Return-Path: Delivered-To: apmail-hadoop-hbase-commits-archive@minotaur.apache.org Received: (qmail 99113 invoked from network); 30 Jul 2009 05:35:43 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 30 Jul 2009 05:35:43 -0000 Received: (qmail 25951 invoked by uid 500); 30 Jul 2009 05:35:44 -0000 Delivered-To: apmail-hadoop-hbase-commits-archive@hadoop.apache.org Received: (qmail 25902 invoked by uid 500); 30 Jul 2009 05:35:44 -0000 Mailing-List: contact hbase-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hbase-dev@hadoop.apache.org Delivered-To: mailing list hbase-commits@hadoop.apache.org Received: (qmail 25892 invoked by uid 99); 30 Jul 2009 05:35:44 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 30 Jul 2009 05:35:44 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 30 Jul 2009 05:35:33 +0000 Received: by eris.apache.org (Postfix, from userid 65534) id 0935323889BE; Thu, 30 Jul 2009 05:35:11 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r799173 [4/4] - in /hadoop/hbase/trunk_on_hadoop-0.18.3: ./ bin/ src/contrib/transactional/ src/contrib/transactional/src/java/org/apache/hadoop/hbase/client/transactional/ src/contrib/transactional/src/java/org/apache/hadoop/hbase/regionse... Date: Thu, 30 Jul 2009 05:35:08 -0000 To: hbase-commits@hadoop.apache.org From: apurtell@apache.org X-Mailer: svnmailer-1.0.8 Message-Id: <20090730053511.0935323889BE@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java?rev=799173&r1=799172&r2=799173&view=diff ============================================================================== --- hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java (original) +++ hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/regionserver/TestStoreScanner.java Thu Jul 30 05:35:05 2009 @@ -20,25 +20,23 @@ package org.apache.hadoop.hbase.regionserver; -import junit.framework.TestCase; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.KeyValueTestUtil; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.filter.WhileMatchFilter; -import org.apache.hadoop.hbase.filter.*; -import org.apache.hadoop.hbase.util.Bytes; - import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.NavigableSet; import java.util.TreeSet; +import junit.framework.TestCase; + +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.KeyValueTestUtil; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.util.Bytes; public class TestStoreScanner extends TestCase { final byte [] CF = Bytes.toBytes("cf"); - + /** * Test utility for building a NavigableSet for scanners. * @param strCols @@ -67,9 +65,9 @@ Scan scanSpec = new Scan(Bytes.toBytes("R1")); // this only uses maxVersions (default=1) and TimeRange (default=all) StoreScanner scan = - new StoreScanner(scanSpec, CF, Long.MAX_VALUE, - KeyValue.COMPARATOR, getCols("a"), - scanners); + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, + KeyValue.COMPARATOR, getCols("a"), + scanners); List results = new ArrayList(); assertEquals(true, scan.next(results)); @@ -98,9 +96,9 @@ Scan scanSpec = new Scan(Bytes.toBytes("R1")); // this only uses maxVersions (default=1) and TimeRange (default=all) StoreScanner scan = - new StoreScanner(scanSpec, CF, Long.MAX_VALUE, - KeyValue.COMPARATOR, getCols("a"), - scanners); + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, + KeyValue.COMPARATOR, getCols("a"), + scanners); List results = new ArrayList(); scan.next(results); @@ -130,8 +128,8 @@ }; Scan scanSpec = new Scan(Bytes.toBytes("R1")); StoreScanner scan = - new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - getCols("a"), scanners); + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a"), scanners); List results = new ArrayList(); assertFalse(scan.next(results)); @@ -153,9 +151,9 @@ }; Scan scanSpec = new Scan(Bytes.toBytes("R1")); StoreScanner scan = - new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - getCols("a"), scanners); - + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a"), scanners); + List results = new ArrayList(); assertEquals(true, scan.next(results)); assertEquals(0, results.size()); @@ -183,8 +181,8 @@ new KeyValueScanFixture(KeyValue.COMPARATOR, kvs2) }; StoreScanner scan = - new StoreScanner(new Scan(Bytes.toBytes("R1")), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - getCols("a"), scanners); + new StoreScanner(new Scan(Bytes.toBytes("R1")), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a"), scanners); List results = new ArrayList(); // the two put at ts=now will be masked by the 1 delete, and // since the scan default returns 1 version we'll return the newest @@ -211,8 +209,8 @@ }; Scan scanSpec = new Scan(Bytes.toBytes("R1")).setMaxVersions(2); StoreScanner scan = - new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - getCols("a"), scanners); + new StoreScanner(scanSpec, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a"), scanners); List results = new ArrayList(); assertEquals(true, scan.next(results)); assertEquals(2, results.size()); @@ -221,17 +219,17 @@ } public void testWildCardOneVersionScan() throws IOException { - KeyValue [] kvs = new KeyValue [] { - KeyValueTestUtil.create("R1", "cf", "a", 2, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R1", "cf", "b", 1, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.DeleteColumn, "dont-care"), - }; + KeyValue [] kvs = new KeyValue [] { + KeyValueTestUtil.create("R1", "cf", "a", 2, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "b", 1, KeyValue.Type.Put, "dont-care"), + KeyValueTestUtil.create("R1", "cf", "a", 1, KeyValue.Type.DeleteColumn, "dont-care"), + }; KeyValueScanner [] scanners = new KeyValueScanner[] { new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) }; StoreScanner scan = - new StoreScanner(new Scan(Bytes.toBytes("R1")), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - null, scanners); + new StoreScanner(new Scan(Bytes.toBytes("R1")), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + null, scanners); List results = new ArrayList(); assertEquals(true, scan.next(results)); assertEquals(2, results.size()); @@ -261,8 +259,8 @@ new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) }; StoreScanner scan = - new StoreScanner(new Scan().setMaxVersions(2), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - null, scanners); + new StoreScanner(new Scan().setMaxVersions(2), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + null, scanners); List results = new ArrayList(); assertEquals(true, scan.next(results)); assertEquals(5, results.size()); @@ -291,8 +289,8 @@ new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) }; StoreScanner scan = - new StoreScanner(new Scan().setMaxVersions(Integer.MAX_VALUE), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - null, scanners); + new StoreScanner(new Scan().setMaxVersions(Integer.MAX_VALUE), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + null, scanners); List results = new ArrayList(); assertEquals(true, scan.next(results)); assertEquals(0, results.size()); @@ -314,8 +312,8 @@ new KeyValueScanFixture(KeyValue.COMPARATOR, kvs), }; StoreScanner scan = - new StoreScanner(new Scan(), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - null, scanners); + new StoreScanner(new Scan(), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + null, scanners); List results = new ArrayList(); assertEquals(true, scan.next(results)); assertEquals(1, results.size()); @@ -339,9 +337,9 @@ new KeyValueScanFixture(KeyValue.COMPARATOR, kvs) }; StoreScanner scan = - new StoreScanner(new Scan(), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, - getCols("a", "d"), scanners); - + new StoreScanner(new Scan(), CF, Long.MAX_VALUE, KeyValue.COMPARATOR, + getCols("a", "d"), scanners); + List results = new ArrayList(); assertEquals(true, scan.next(results)); assertEquals(2, results.size()); @@ -352,156 +350,8 @@ assertEquals(true, scan.next(results)); assertEquals(1, results.size()); assertEquals(kvs[kvs.length-1], results.get(0)); - - results.clear(); - assertEquals(false, scan.next(results)); - } - - KeyValue [] stdKvs = new KeyValue[] { - KeyValueTestUtil.create("R:1", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:1", "cf", "b", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:1", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:1", "cf", "d", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:1", "cf", "e", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:1", "cf", "f", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:1", "cf", "g", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:1", "cf", "h", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:1", "cf", "i", 11, KeyValue.Type.Put, "dont-care"), - - // 9... - KeyValueTestUtil.create("R:2", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:2", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:2", "cf", "c", 10, KeyValue.Type.Put, "dont-care"), - - // 12... - KeyValueTestUtil.create("R:3", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:3", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:3", "cf", "c", 10, KeyValue.Type.Put, "dont-care"), - - // 15 ... - KeyValueTestUtil.create("R:4", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:4", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:4", "cf", "c", 10, KeyValue.Type.Put, "dont-care"), - - // 18 .. - KeyValueTestUtil.create("R:5", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:5", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), - - // 20... - KeyValueTestUtil.create("R:6", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:6", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), - - // 22... - KeyValueTestUtil.create("R:7", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:7", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), - - // 24... - KeyValueTestUtil.create("R:8", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - KeyValueTestUtil.create("R:8", "cf", "c", 11, KeyValue.Type.Put, "dont-care"), - - // 26 .. - KeyValueTestUtil.create("RA:1", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - - // 27... - KeyValueTestUtil.create("RA:2", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - - // 28.. - KeyValueTestUtil.create("RA:3", "cf", "a", 11, KeyValue.Type.Put, "dont-care"), - }; - private StoreScanner getTestScanner(Scan s, NavigableSet cols) { - KeyValueScanner [] scanners = new KeyValueScanner[] { - new KeyValueScanFixture(KeyValue.COMPARATOR, stdKvs) - }; - - return new StoreScanner(s, CF, Long.MAX_VALUE, KeyValue.COMPARATOR, cols, - scanners); - } - - - // Test new and old row prefix filters. - public void testNewRowPrefixFilter() throws IOException { - Filter f = new WhileMatchFilter( - new PrefixFilter(Bytes.toBytes("R:"))); - Scan s = new Scan(Bytes.toBytes("R:7")); - s.setFilter(f); - rowPrefixFilter(s); - } - - public void testOldRowPrefixFilter() throws IOException { - RowFilterInterface f = new WhileMatchRowFilter( - new PrefixRowFilter(Bytes.toBytes("R:"))); - Scan s = new Scan(Bytes.toBytes("R:7")); - s.setOldFilter(f); - - rowPrefixFilter(s); - - } - public void rowPrefixFilter(Scan s) throws IOException { - - StoreScanner scan = getTestScanner(s, null); - - List results = new ArrayList(); - assertTrue(scan.next(results)); - assertEquals(2, results.size()); - assertEquals(stdKvs[22], results.get(0)); - assertEquals(stdKvs[23], results.get(1)); - results.clear(); - - assertTrue(scan.next(results)); - assertEquals(2, results.size()); - assertEquals(stdKvs[24], results.get(0)); - assertEquals(stdKvs[25], results.get(1)); - results.clear(); - - assertFalse(scan.next(results)); - assertEquals(0, results.size()); - } - - // Test new and old row-inclusive stop filter. - public void testNewRowInclusiveStopFilter() throws IOException { - Filter f = new WhileMatchFilter(new InclusiveStopFilter(Bytes.toBytes("R:3"))); - Scan scan = new Scan(); - scan.setFilter(f); - - rowInclusiveStopFilter(scan); - } - - public void testOldRowInclusiveTopFilter() throws IOException { - RowFilterInterface f = new WhileMatchRowFilter( - new InclusiveStopRowFilter(Bytes.toBytes("R:3"))); - Scan scan = new Scan(); - scan.setOldFilter(f); - - rowInclusiveStopFilter(scan); - } - - public void rowInclusiveStopFilter(Scan scan) throws IOException { - StoreScanner s = getTestScanner(scan, getCols("a")); - - // read crap. - List results = new ArrayList(); - assertTrue(s.next(results)); - assertEquals(1, results.size()); - assertEquals(stdKvs[0], results.get(0)); - results.clear(); - - assertTrue(s.next(results)); - assertEquals(1, results.size()); - assertEquals(stdKvs[9], results.get(0)); results.clear(); - - assertTrue(s.next(results)); - assertEquals(1, results.size()); - assertEquals(stdKvs[12], results.get(0)); - results.clear(); - - // without aggressive peeking, the scanner doesnt know if the next row is good or not - // under the affects of a filter. - assertFalse(s.next(results)); - assertEquals(0, results.size()); + assertEquals(false, scan.next(results)); } - - - } Modified: hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java?rev=799173&r1=799172&r2=799173&view=diff ============================================================================== --- hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java (original) +++ hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/thrift/TestThriftServer.java Thu Jul 30 05:35:05 2009 @@ -322,7 +322,7 @@ // A slightly customized ColumnDescriptor (only 2 versions) ColumnDescriptor cDescB = new ColumnDescriptor(columnBname, 2, "NONE", - false, 2147483647, "NONE", 0, 0, false, -1); + false, "NONE", 0, 0, false, -1); cDescriptors.add(cDescB); return cDescriptors; Added: hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/util/TestMigration.java URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/util/TestMigration.java?rev=799173&view=auto ============================================================================== --- hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/util/TestMigration.java (added) +++ hadoop/hbase/trunk_on_hadoop-0.18.3/src/test/org/apache/hadoop/hbase/util/TestMigration.java Thu Jul 30 05:35:05 2009 @@ -0,0 +1,253 @@ +/** + * 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.util; + +import java.io.IOException; +import java.io.InputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipInputStream; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hbase.HBaseTestCase; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MiniHBaseCluster; +import org.apache.hadoop.hbase.MiniZooKeeperCluster; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HBaseAdmin; +import org.apache.hadoop.hbase.client.HConnectionManager; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.ResultScanner; +import org.apache.hadoop.hbase.client.Scan; + +/** + * Runs migration of filesystem from hbase 0.19 to hbase 0.20. + * Not part of general test suite because takes time. + */ +public class TestMigration extends HBaseTestCase { + private static final Log LOG = LogFactory.getLog(TestMigration.class); + + // Expected count of rows in migrated table. + private static final int EXPECTED_COUNT = 3; + + /** + * Test migration. + * @throws IOException + * @throws InterruptedException + */ + public void testMigration() throws IOException, InterruptedException { + Path rootdir = getUnitTestdir(getName()); + Path hbasedir = loadTestData(fs, rootdir); + assertTrue(fs.exists(hbasedir)); + Migrate migrator = new Migrate(this.conf); + Path qualified = fs.makeQualified(hbasedir); + String uri = qualified.toString(); + this.conf.set("hbase.rootdir", uri); + int result = migrator.run(new String [] {"upgrade"}); + assertEquals(0, result); + verify(); + } + + /* + * Load up test data. + * @param dfs + * @param rootDir + * @throws IOException + */ + private Path loadTestData(final FileSystem dfs, final Path rootDir) + throws IOException { + String hbasedir = "hbase-0.19-two-small-tables"; + InputStream is = this.getClass().getClassLoader(). + getResourceAsStream("data/" + hbasedir + ".zip"); + ZipInputStream zip = new ZipInputStream(is); + try { + unzip(zip, dfs, rootDir); + } finally { + zip.close(); + } + return new Path(rootDir, hbasedir); + } + + /* + * Verify can read the migrated table. + * @throws IOException + */ + private void verify() throws IOException, InterruptedException { + // Delete any cached connections. Need to do this because connection was + // created earlier when no master was around. The fact that there was no + // master gets cached. Need to delete so we go get master afresh. + HConnectionManager.deleteConnectionInfo(conf, false); + LOG.info("Start a cluster against migrated FS"); + // Up number of retries. Needed while cluster starts up. Its been set to 1 + // above. + final int retries = 5; + this.conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER_KEY, retries); + // Note that this is done before we create the MiniHBaseCluster because we + // need to edit the config to add the ZooKeeper servers. + MiniZooKeeperCluster zooKeeperCluster = new MiniZooKeeperCluster(); + int clientPort = + zooKeeperCluster.startup(new java.io.File(this.testDir.toString())); + conf.set("hbase.zookeeper.property.clientPort", + Integer.toString(clientPort)); + MiniHBaseCluster cluster = new MiniHBaseCluster(this.conf, 1); + try { + HBaseAdmin hb = new HBaseAdmin(this.conf); + assertTrue(hb.isMasterRunning()); + HTableDescriptor [] tables = hb.listTables(); + assertEquals(2, tables.length); + boolean foundTable = false; + // Just look at table 'a'. + final String tablenameStr = "a"; + final byte [] tablename = Bytes.toBytes(tablenameStr); + for (int i = 0; i < tables.length; i++) { + byte [] tableName = tables[i].getName(); + if (Bytes.equals(tablename, tables[i].getName())) { + foundTable = true; + break; + } + } + assertTrue(foundTable); + LOG.info(tablenameStr + " exists. Now waiting till startcode " + + "changes before opening a scanner"); + waitOnStartCodeChange(retries); + // Delete again so we go get it all fresh. + HConnectionManager.deleteConnectionInfo(conf, false); + HTable t = new HTable(this.conf, tablename); + int count = 0; + LOG.info("OPENING SCANNER"); + Scan scan = new Scan(); + ResultScanner s = t.getScanner(scan); + try { + for (Result r: s) { + if (r == null || r.size() == 0) { + break; + } + count++; + } + assertEquals(EXPECTED_COUNT, count); + } finally { + s.close(); + } + } finally { + HConnectionManager.deleteConnectionInfo(conf, false); + cluster.shutdown(); + try { + zooKeeperCluster.shutdown(); + } catch (IOException e) { + LOG.warn("Shutting down ZooKeeper cluster", e); + } + } + } + + /* + * Wait till the startcode changes before we put up a scanner. Otherwise + * we tend to hang, at least on hudson and I've had it time to time on + * my laptop. The hang is down in RPC Client doing its call. It + * never returns though the socket has a read timeout of 60 seconds by + * default. St.Ack + * @param retries How many retries to run. + * @throws IOException + */ + private void waitOnStartCodeChange(final int retries) throws IOException { + HTable m = new HTable(this.conf, HConstants.META_TABLE_NAME); + // This is the start code that is in the old data. + long oldStartCode = 1199736332062L; + // This is the first row for the TestTable that is in the old data. + byte [] row = Bytes.toBytes("TestUpgrade,,1199736362468"); + long pause = conf.getLong("hbase.client.pause", 5 * 1000); + long startcode = -1; + boolean changed = false; + for (int i = 0; i < retries; i++) { + Get get = new Get(row); + get.addColumn(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER); + Result res = m.get(get); + KeyValue [] kvs = res.raw(); + if(kvs.length <= 0){ + return; + } + byte [] value = kvs[0].getValue(); + startcode = Bytes.toLong(value); + if (startcode != oldStartCode) { + changed = true; + break; + } + if ((i + 1) != retries) { + try { + Thread.sleep(pause); + } catch (InterruptedException e) { + // continue + } + } + } + // If after all attempts startcode has not changed, fail. + if (!changed) { + throw new IOException("Startcode didn't change after " + retries + + " attempts"); + } + } + + private void unzip(ZipInputStream zip, FileSystem dfs, Path rootDir) + throws IOException { + ZipEntry e = null; + while ((e = zip.getNextEntry()) != null) { + if (e.isDirectory()) { + dfs.mkdirs(new Path(rootDir, e.getName())); + } else { + FSDataOutputStream out = dfs.create(new Path(rootDir, e.getName())); + byte[] buffer = new byte[4096]; + int len; + do { + len = zip.read(buffer); + if (len > 0) { + out.write(buffer, 0, len); + } + } while (len > 0); + out.close(); + } + zip.closeEntry(); + } + } + + @SuppressWarnings("unused") + private void listPaths(FileSystem filesystem, Path dir, int rootdirlength) + throws IOException { + FileStatus[] stats = filesystem.listStatus(dir); + if (stats == null || stats.length == 0) { + return; + } + for (int i = 0; i < stats.length; i++) { + String path = stats[i].getPath().toString(); + if (stats[i].isDir()) { + System.out.println("d " + path); + listPaths(filesystem, stats[i].getPath(), rootdirlength); + } else { + System.out.println("f " + path + " size=" + stats[i].getLen()); + } + } + } +} \ No newline at end of file Modified: hadoop/hbase/trunk_on_hadoop-0.18.3/src/webapps/master/master.jsp URL: http://svn.apache.org/viewvc/hadoop/hbase/trunk_on_hadoop-0.18.3/src/webapps/master/master.jsp?rev=799173&r1=799172&r2=799173&view=diff ============================================================================== --- hadoop/hbase/trunk_on_hadoop-0.18.3/src/webapps/master/master.jsp (original) +++ hadoop/hbase/trunk_on_hadoop-0.18.3/src/webapps/master/master.jsp Thu Jul 30 05:35:05 2009 @@ -30,19 +30,7 @@ HBase Master: <%= master.getMasterAddress().getHostname()%>:<%= master.getMasterAddress().getPort() %> - - - - - - @@ -80,56 +68,12 @@

User Tables

<% HTableDescriptor[] tables = new HBaseAdmin(conf).listTables(); if(tables != null && tables.length > 0) { %> - -
-
    -<% for(HTableDescriptor htDesc : tables) { %> -
  •  <%= htDesc.getNameAsString() %> -
      -
    •  Parameters -
        -<% Map vals = htDesc.getValues(); - if (vals.size() > 0) { - for (Map.Entry e: vals.entrySet()) { %> -
      •   <%= Bytes.toString(e.getKey().get()).toLowerCase() %>: <%= Bytes.toString(e.getValue().get()).toLowerCase() %>
      • -<% } - } else { %> -
      •   none
      • -<% } %> -
      -
    • - -
    •  Families -
        -<% Collection cols = htDesc.getFamilies(); - if (cols.size() > 0) { - for (HColumnDescriptor hcd: htDesc.getFamilies()) { %> -
      •  Name: <%= hcd.getNameAsString() %> -
          -<% for (Map.Entry e: hcd.getValues().entrySet()) { %> -
        •   <%= Bytes.toString(e.getKey().get()).toLowerCase() %>: <%= Bytes.toString(e.getValue().get()).toLowerCase() %>
        • -<% } %> -
        -<% } %> -
      • -<% } else { %> -
      •  none
      • -<% }%> -
      -
    • - -
    -
  • - - + + +<% for(HTableDescriptor htDesc : tables ) { %> + <% } %> - -

    <%= tables.length %> table(s) in set.

    TableDescription
    ><%= htDesc.getNameAsString() %> <%= htDesc.toString() %>
    <% } %>