Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 7948C9C90 for ; Fri, 13 Apr 2012 17:21:45 +0000 (UTC) Received: (qmail 53693 invoked by uid 500); 13 Apr 2012 17:21:45 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 53637 invoked by uid 500); 13 Apr 2012 17:21:45 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 53626 invoked by uid 99); 13 Apr 2012 17:21:44 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 13 Apr 2012 17:21:44 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.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; Fri, 13 Apr 2012 17:21:32 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id BAC152388C1C for ; Fri, 13 Apr 2012 17:21:09 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1325853 - in /accumulo/branches/1.4/src: core/src/main/java/org/apache/accumulo/core/client/impl/ core/src/main/java/org/apache/accumulo/core/file/ core/src/main/java/org/apache/accumulo/core/file/map/ core/src/main/java/org/apache/accumul... Date: Fri, 13 Apr 2012 17:21:08 -0000 To: commits@accumulo.apache.org From: kturner@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20120413172109.BAC152388C1C@eris.apache.org> Author: kturner Date: Fri Apr 13 17:21:07 2012 New Revision: 1325853 URL: http://svn.apache.org/viewvc?rev=1325853&view=rev Log: ACCUMULO-516 Moved column family filter from below heap iterator to above it Removed: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/FileCFSkippingIterator.java Modified: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MyMapFile.java accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Compactor.java accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/NativeMap.java accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/SparseColumnFamilyTest.java accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java accumulo/branches/1.4/src/server/src/test/java/org/apache/accumulo/server/tabletserver/InMemoryMapTest.java Modified: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java (original) +++ accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/client/impl/OfflineScanner.java Fri Apr 13 17:21:07 2012 @@ -47,6 +47,7 @@ import org.apache.accumulo.core.iterator import org.apache.accumulo.core.iterators.IteratorUtil; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator; import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter; import org.apache.accumulo.core.iterators.system.DeletingIterator; import org.apache.accumulo.core.iterators.system.MultiIterator; @@ -313,7 +314,9 @@ class OfflineIterator implements Iterato DeletingIterator delIter = new DeletingIterator(multiIter, false); - ColumnQualifierFilter colFilter = new ColumnQualifierFilter(delIter, new HashSet(options.fetchedColumns)); + ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter); + + ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, new HashSet(options.fetchedColumns)); byte[] defaultSecurityLabel; Modified: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java (original) +++ accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MapFileOperations.java Fri Apr 13 17:21:07 2012 @@ -30,7 +30,6 @@ import org.apache.accumulo.core.data.Byt import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.file.FileCFSkippingIterator; import org.apache.accumulo.core.file.FileOperations; import org.apache.accumulo.core.file.FileSKVIterator; import org.apache.accumulo.core.file.FileSKVWriter; @@ -137,7 +136,7 @@ public class MapFileOperations extends F @Override public FileSKVIterator openReader(String file, boolean seekToBeginning, FileSystem fs, Configuration conf, AccumuloConfiguration acuconf) throws IOException { - FileSKVIterator iter = new FileCFSkippingIterator(new RangeIterator(MapFileUtil.openMapFile(acuconf, fs, file, conf))); + RangeIterator iter = new RangeIterator(MapFileUtil.openMapFile(acuconf, fs, file, conf)); if (seekToBeginning) iter.seek(new Range(new Key(), null), new ArrayList(), false); @@ -208,10 +207,6 @@ public class MapFileOperations extends F FileSKVIterator iter = new RangeIterator(mfIter); - if (columnFamilies.size() != 0 || inclusive) { - iter = new FileCFSkippingIterator(iter); - } - iter.seek(range, columnFamilies, inclusive); mfIter.dropIndex(); Modified: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MyMapFile.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MyMapFile.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MyMapFile.java (original) +++ accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/map/MyMapFile.java Fri Apr 13 17:21:07 2012 @@ -809,10 +809,6 @@ public class MyMapFile { public void seek(Range range, Collection columnFamilies, boolean inclusive) throws IOException { - if (columnFamilies.size() != 0 || inclusive) { - throw new IllegalArgumentException("I do not know how to filter column families"); - } - if (range == null) throw new IllegalArgumentException("Cannot seek to null range"); Modified: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java (original) +++ accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/file/rfile/RFile.java Fri Apr 13 17:21:07 2012 @@ -44,7 +44,6 @@ import org.apache.accumulo.core.data.Byt import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.file.FileCFSkippingIterator; import org.apache.accumulo.core.file.FileSKVIterator; import org.apache.accumulo.core.file.FileSKVWriter; import org.apache.accumulo.core.file.NoSuchMetaStoreException; @@ -967,17 +966,12 @@ public class RFile { for (LocalityGroupReader lgr : lgReaders) { - // when exclude is set to true it means this locality group contains - // unwanted column families - boolean exclude = false; - // when include is set to true it means this locality groups contains // wanted column families boolean include = false; if (cfSet.size() == 0) { include = !inclusive; - exclude = false; } else if (lgr.isDefaultLocalityGroup && lgr.columnFamilies == null) { // do not know what column families are in the default locality group, // only know what column families are not in it @@ -985,14 +979,12 @@ public class RFile { if (inclusive) { if (!nonDefaultColumnFamilies.containsAll(cfSet)) { // default LG may contain wanted and unwanted column families - exclude = true; include = true; }// else - everything wanted is in other locality groups, so nothing to do } else { // must include, if all excluded column families are in other locality groups // then there are not unwanted column families in default LG include = true; - exclude = !nonDefaultColumnFamilies.containsAll(cfSet); } } else { /* @@ -1002,28 +994,17 @@ public class RFile { for (Entry entry : lgr.columnFamilies.entrySet()) if (entry.getValue().count > 0) - if (cfSet.contains(entry.getKey())) + if (cfSet.contains(entry.getKey())) { if (inclusive) include = true; - else - exclude = true; - else if (inclusive) - exclude = true; - else + } else if (!inclusive) { include = true; + } } if (include) { - if (exclude) { - // want a subset of what is in the locality group, therefore filtering is need - FileCFSkippingIterator cfe = new FileCFSkippingIterator(lgr); - cfe.seek(range, cfSet, inclusive); - addSource(cfe); - } else { - // want everything in this locality group, therefore no filtering is needed - lgr.seek(range, EMPTY_CF_SET, false); - addSource(lgr); - } + lgr.seek(range, EMPTY_CF_SET, false); + addSource(lgr); numLGSeeked++; }// every column family is excluded, zero count, or not present } Modified: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java (original) +++ accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/SortedMapIterator.java Fri Apr 13 17:21:07 2012 @@ -20,8 +20,8 @@ import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.Map; -import java.util.SortedMap; import java.util.Map.Entry; +import java.util.SortedMap; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.accumulo.core.data.ByteSequence; @@ -100,10 +100,6 @@ public class SortedMapIterator implement @Override public void seek(Range range, Collection columnFamilies, boolean inclusive) throws IOException { - if (columnFamilies.size() != 0 || inclusive) { - throw new IllegalArgumentException("I do not know how to filter column families"); - } - if (interruptFlag != null && interruptFlag.get()) throw new IterationInterruptedException(); Modified: accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java (original) +++ accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/system/ColumnFamilySkippingIterator.java Fri Apr 13 17:21:07 2012 @@ -92,10 +92,10 @@ public class ColumnFamilySkippingIterato private void reseek(Key key) throws IOException { if (range.afterEndKey(key)) { range = new Range(range.getEndKey(), true, range.getEndKey(), range.isEndKeyInclusive()); - getSource().seek(range, EMPTY_SET, false); + getSource().seek(range, colFamSet, inclusive); } else { range = new Range(key, true, range.getEndKey(), range.isEndKeyInclusive()); - getSource().seek(range, EMPTY_SET, false); + getSource().seek(range, colFamSet, inclusive); } } @@ -122,7 +122,7 @@ public class ColumnFamilySkippingIterato this.range = range; this.inclusive = inclusive; - super.seek(range, EMPTY_SET, false); + super.seek(range, colFamSet, inclusive); } @Override Modified: accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java (original) +++ accumulo/branches/1.4/src/core/src/test/java/org/apache/accumulo/core/file/rfile/RFileTest.java Fri Apr 13 17:21:07 2012 @@ -38,9 +38,9 @@ import org.apache.accumulo.core.data.Ran import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.file.FileSKVIterator; import org.apache.accumulo.core.file.blockfile.impl.CachableBlockFile; -import org.apache.accumulo.core.file.rfile.RFile; import org.apache.accumulo.core.file.rfile.RFile.Reader; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator; import org.apache.accumulo.core.util.CachedConfiguration; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; @@ -151,7 +151,8 @@ public class RFileTest extends TestCase private SeekableByteArrayInputStream bais; private FSDataInputStream in; public Reader reader; - + public SortedKeyValueIterator iter; + public void openWriter(boolean startDLG) throws IOException { baos = new ByteArrayOutputStream(); dos = new FSDataOutputStream(baos, new FileSystem.Statistics("a")); @@ -178,6 +179,7 @@ public class RFileTest extends TestCase in = new FSDataInputStream(bais); CachableBlockFile.Reader _cbr = new CachableBlockFile.Reader(in, data.length, conf); reader = new RFile.Reader(_cbr); + iter = new ColumnFamilySkippingIterator(reader); checkIndex(reader); } @@ -188,7 +190,7 @@ public class RFileTest extends TestCase } public void seek(Key nk) throws IOException { - reader.seek(new Range(nk, null), EMPTY_COL_FAMS, false); + iter.seek(new Range(nk, null), EMPTY_COL_FAMS, false); } } @@ -214,8 +216,8 @@ public class RFileTest extends TestCase trf.closeWriter(); trf.openReader(); - trf.reader.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false); + assertFalse(trf.iter.hasTop()); assertEquals(null, trf.reader.getLastKey()); @@ -235,23 +237,23 @@ public class RFileTest extends TestCase trf.openReader(); // seek before everything trf.seek(null); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("r1", "cf1", "cq1", "L1", 55))); - assertTrue(trf.reader.getTopValue().equals(nv("foo"))); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("r1", "cf1", "cq1", "L1", 55))); + assertTrue(trf.iter.getTopValue().equals(nv("foo"))); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // seek after the key trf.seek(nk("r2", "cf1", "cq1", "L1", 55)); - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); // seek exactly to the key trf.seek(nk("r1", "cf1", "cq1", "L1", 55)); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("r1", "cf1", "cq1", "L1", 55))); - assertTrue(trf.reader.getTopValue().equals(nv("foo"))); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("r1", "cf1", "cq1", "L1", 55))); + assertTrue(trf.iter.getTopValue().equals(nv("foo"))); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); assertEquals(nk("r1", "cf1", "cq1", "L1", 55), trf.reader.getLastKey()); @@ -305,7 +307,7 @@ public class RFileTest extends TestCase trf.openReader(); // seek before everything - trf.reader.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false); + trf.iter.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false); verify(trf, expectedKeys.iterator(), expectedValues.iterator()); // seek to the middle @@ -331,29 +333,29 @@ public class RFileTest extends TestCase // test seeking to the current location index = expectedKeys.size() / 2; trf.seek(expectedKeys.get(index)); - assertTrue(trf.reader.hasTop()); - assertEquals(expectedKeys.get(index), trf.reader.getTopKey()); - assertEquals(expectedValues.get(index), trf.reader.getTopValue()); + assertTrue(trf.iter.hasTop()); + assertEquals(expectedKeys.get(index), trf.iter.getTopKey()); + assertEquals(expectedValues.get(index), trf.iter.getTopValue()); - trf.reader.next(); + trf.iter.next(); index++; - assertTrue(trf.reader.hasTop()); - assertEquals(expectedKeys.get(index), trf.reader.getTopKey()); - assertEquals(expectedValues.get(index), trf.reader.getTopValue()); + assertTrue(trf.iter.hasTop()); + assertEquals(expectedKeys.get(index), trf.iter.getTopKey()); + assertEquals(expectedValues.get(index), trf.iter.getTopValue()); trf.seek(expectedKeys.get(index)); - assertTrue(trf.reader.hasTop()); - assertEquals(expectedKeys.get(index), trf.reader.getTopKey()); - assertEquals(expectedValues.get(index), trf.reader.getTopValue()); + assertTrue(trf.iter.hasTop()); + assertEquals(expectedKeys.get(index), trf.iter.getTopKey()); + assertEquals(expectedValues.get(index), trf.iter.getTopValue()); // test seeking to each location in the file index = 0; for (Key key : expectedKeys) { trf.seek(key); - assertTrue(trf.reader.hasTop()); - assertEquals(key, trf.reader.getTopKey()); - assertEquals(expectedValues.get(index), trf.reader.getTopValue()); + assertTrue(trf.iter.hasTop()); + assertEquals(key, trf.iter.getTopKey()); + assertEquals(expectedValues.get(index), trf.iter.getTopValue()); if (index > 0) { // Key pkey = @@ -369,9 +371,9 @@ public class RFileTest extends TestCase Key key = expectedKeys.get(i); trf.seek(key); - assertTrue(trf.reader.hasTop()); - assertEquals(key, trf.reader.getTopKey()); - assertEquals(expectedValues.get(i), trf.reader.getTopValue()); + assertTrue(trf.iter.hasTop()); + assertEquals(key, trf.iter.getTopKey()); + assertEquals(expectedValues.get(i), trf.iter.getTopValue()); if (i - 1 > 0) { // Key pkey = @@ -387,14 +389,14 @@ public class RFileTest extends TestCase private void verify(TestRFile trf, Iterator eki, Iterator evi) throws IOException { - while (trf.reader.hasTop()) { + while (trf.iter.hasTop()) { Key ek = eki.next(); Value ev = evi.next(); - assertEquals(ek, trf.reader.getTopKey()); - assertEquals(ev, trf.reader.getTopValue()); + assertEquals(ek, trf.iter.getTopKey()); + assertEquals(ev, trf.iter.getTopValue()); - trf.reader.next(); + trf.iter.next(); } assertFalse(eki.hasNext()); @@ -456,15 +458,15 @@ public class RFileTest extends TestCase // test seeking between keys trf.seek(nk("r1", "cf1", "cq3", "L1", 55)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk("r1", "cf1", "cq4", "L1", 56), trf.reader.getTopKey()); - assertEquals(nv("foo2"), trf.reader.getTopValue()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("r1", "cf1", "cq4", "L1", 56), trf.iter.getTopKey()); + assertEquals(nv("foo2"), trf.iter.getTopValue()); // test seeking right before previous seek trf.seek(nk("r1", "cf1", "cq0", "L1", 55)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk("r1", "cf1", "cq1", "L1", 55), trf.reader.getTopKey()); - assertEquals(nv("foo1"), trf.reader.getTopValue()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("r1", "cf1", "cq1", "L1", 55), trf.iter.getTopKey()); + assertEquals(nv("foo1"), trf.iter.getTopValue()); assertEquals(nk("r1", "cf1", "cq4", "L1", 56), trf.reader.getLastKey()); @@ -487,15 +489,15 @@ public class RFileTest extends TestCase // repeatedly seek to locations before the first key in the file for (int i = 0; i < 10; i++) { trf.seek(nk(nf("q_", i), "cf1", "cq1", "L1", 55)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", 0), "cf1", "cq1", "L1", 55), trf.reader.getTopKey()); - assertEquals(nv("foo1"), trf.reader.getTopValue()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", 0), "cf1", "cq1", "L1", 55), trf.iter.getTopKey()); + assertEquals(nv("foo1"), trf.iter.getTopValue()); } // repeatedly seek to locations after the last key in the file for (int i = 0; i < 10; i++) { trf.seek(nk(nf("s_", i), "cf1", "cq1", "L1", 55)); - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); } assertEquals(nk(nf("r_", 499), "cf1", "cq1", "L1", 55), trf.reader.getLastKey()); @@ -518,36 +520,36 @@ public class RFileTest extends TestCase trf.openReader(); // test that has top returns false when end of range reached - trf.reader.seek(new Range(nk(nf("r_", 3), "cf1", "cq1", "L1", 55), true, nk(nf("r_", 4), "cf1", "cq1", "L1", 55), false), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk(nf("r_", 3), "cf1", "cq1", "L1", 55))); - assertEquals(nv("foo" + 3), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(nk(nf("r_", 3), "cf1", "cq1", "L1", 55), true, nk(nf("r_", 4), "cf1", "cq1", "L1", 55), false), EMPTY_COL_FAMS, false); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk(nf("r_", 3), "cf1", "cq1", "L1", 55))); + assertEquals(nv("foo" + 3), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // test seeking to a range that is between two keys, should not return anything - trf.reader.seek(new Range(nk(nf("r_", 4) + "a", "cf1", "cq1", "L1", 55), true, nk(nf("r_", 4) + "b", "cf1", "cq1", "L1", 55), true), EMPTY_COL_FAMS, false); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(nk(nf("r_", 4) + "a", "cf1", "cq1", "L1", 55), true, nk(nf("r_", 4) + "b", "cf1", "cq1", "L1", 55), true), EMPTY_COL_FAMS, false); + assertFalse(trf.iter.hasTop()); // test seeking to another range after the previously seeked range, that is between the same two keys in the file // as the previously seeked range.... this test an optimization on RFile - trf.reader.seek(new Range(nk(nf("r_", 4) + "c", "cf1", "cq1", "L1", 55), true, nk(nf("r_", 4) + "d", "cf1", "cq1", "L1", 55), true), EMPTY_COL_FAMS, false); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(nk(nf("r_", 4) + "c", "cf1", "cq1", "L1", 55), true, nk(nf("r_", 4) + "d", "cf1", "cq1", "L1", 55), true), EMPTY_COL_FAMS, false); + assertFalse(trf.iter.hasTop()); - trf.reader.seek(new Range(nk(nf("r_", 4) + "e", "cf1", "cq1", "L1", 55), true, nk(nf("r_", 4) + "f", "cf1", "cq1", "L1", 55), true), EMPTY_COL_FAMS, false); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(nk(nf("r_", 4) + "e", "cf1", "cq1", "L1", 55), true, nk(nf("r_", 4) + "f", "cf1", "cq1", "L1", 55), true), EMPTY_COL_FAMS, false); + assertFalse(trf.iter.hasTop()); // now ensure we can seek somewhere, that triggering the optimization does not cause any problems - trf.reader.seek(new Range(nk(nf("r_", 5), "cf1", "cq1", "L1", 55), true, nk(nf("r_", 6), "cf1", "cq1", "L1", 55), false), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk(nf("r_", 5), "cf1", "cq1", "L1", 55))); - assertEquals(nv("foo" + 5), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(nk(nf("r_", 5), "cf1", "cq1", "L1", 55), true, nk(nf("r_", 6), "cf1", "cq1", "L1", 55), false), EMPTY_COL_FAMS, false); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk(nf("r_", 5), "cf1", "cq1", "L1", 55))); + assertEquals(nv("foo" + 5), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // test seeking to range that is before the beginning of the file - trf.reader.seek(new Range(nk(nf("r_", 0), "cf1", "cq1", "L1", 55), true, nk(nf("r_", 2), "cf1", "cq1", "L1", 55), false), EMPTY_COL_FAMS, false); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(nk(nf("r_", 0), "cf1", "cq1", "L1", 55), true, nk(nf("r_", 2), "cf1", "cq1", "L1", 55), false), EMPTY_COL_FAMS, false); + assertFalse(trf.iter.hasTop()); assertEquals(nk(nf("r_", 49), "cf1", "cq1", "L1", 55), trf.reader.getLastKey()); @@ -569,22 +571,22 @@ public class RFileTest extends TestCase // test seeking between each key forward for (int i = 0; i < 2499; i++) { trf.seek(nk(nf("r_", i), "cf1", "cq1", "L1", 42).followingKey(PartialKey.ROW)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", i + 1), "cf1", "cq1", "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", i + 1), "cf1", "cq1", "L1", 42), trf.iter.getTopKey()); } // test seeking between each key forward for (int i = 0; i < 2499; i += 2) { trf.seek(nk(nf("r_", i), "cf1", "cq1", "L1", 42).followingKey(PartialKey.ROW)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", i + 1), "cf1", "cq1", "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", i + 1), "cf1", "cq1", "L1", 42), trf.iter.getTopKey()); } // test seeking backwards between each key for (int i = 2498; i >= 0; i--) { trf.seek(nk(nf("r_", i), "cf1", "cq1", "L1", 42).followingKey(PartialKey.ROW)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", i + 1), "cf1", "cq1", "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", i + 1), "cf1", "cq1", "L1", 42), trf.iter.getTopKey()); } trf.closeReader(); @@ -604,22 +606,22 @@ public class RFileTest extends TestCase // test seeking between each key forward for (int i = 0; i < 2499; i++) { trf.seek(nk(nf("r_", 0), nf("cf_", i), "cq1", "L1", 42).followingKey(PartialKey.ROW_COLFAM)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", 0), nf("cf_", i + 1), "cq1", "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", 0), nf("cf_", i + 1), "cq1", "L1", 42), trf.iter.getTopKey()); } // test seeking between each key forward for (int i = 0; i < 2499; i += 2) { trf.seek(nk(nf("r_", 0), nf("cf_", i), "cq1", "L1", 42).followingKey(PartialKey.ROW_COLFAM)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", 0), nf("cf_", i + 1), "cq1", "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", 0), nf("cf_", i + 1), "cq1", "L1", 42), trf.iter.getTopKey()); } // test seeking backwards between each key for (int i = 2498; i >= 0; i--) { trf.seek(nk(nf("r_", 0), nf("cf_", i), "cq1", "L1", 42).followingKey(PartialKey.ROW_COLFAM)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", 0), nf("cf_", i + 1), "cq1", "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", 0), nf("cf_", i + 1), "cq1", "L1", 42), trf.iter.getTopKey()); } trf.closeReader(); @@ -639,22 +641,22 @@ public class RFileTest extends TestCase // test seeking between each key forward for (int i = 0; i < 2499; i++) { trf.seek(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i), "L1", 42).followingKey(PartialKey.ROW_COLFAM_COLQUAL)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i + 1), "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i + 1), "L1", 42), trf.iter.getTopKey()); } // test seeking between each key forward for (int i = 0; i < 2499; i += 2) { trf.seek(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i), "L1", 42).followingKey(PartialKey.ROW_COLFAM_COLQUAL)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i + 1), "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i + 1), "L1", 42), trf.iter.getTopKey()); } // test seeking backwards between each key for (int i = 2498; i >= 0; i--) { trf.seek(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i), "L1", 42).followingKey(PartialKey.ROW_COLFAM_COLQUAL)); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i + 1), "L1", 42), trf.reader.getTopKey()); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("r_", 0), nf("cf_", 0), nf("cq_", i + 1), "L1", 42), trf.iter.getTopKey()); } trf.closeReader(); @@ -691,121 +693,121 @@ public class RFileTest extends TestCase // scan first loc group Range r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, ncfs("cf1", "cf2"), true); + trf.iter.seek(r, ncfs("cf1", "cf2"), true); assertEquals(1, trf.reader.getNumLocalityGroupsSeeked()); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); - assertEquals(nv("1123 West Left st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); - assertEquals(nv("1124 East Right st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); + assertEquals(nv("1123 West Left st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); + assertEquals(nv("1124 East Right st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // scan second loc group r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, ncfs("cf3", "cf4"), true); + trf.iter.seek(r, ncfs("cf3", "cf4"), true); assertEquals(1, trf.reader.getNumLocalityGroupsSeeked()); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0001", "cf3", "buck,john", "", 4))); - assertEquals(nv("90 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); - assertEquals(nv("09 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0001", "cf3", "buck,john", "", 4))); + assertEquals(nv("90 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); + assertEquals(nv("09 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // scan all loc groups r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, EMPTY_COL_FAMS, false); + trf.iter.seek(r, EMPTY_COL_FAMS, false); assertEquals(2, trf.reader.getNumLocalityGroupsSeeked()); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); - assertEquals(nv("1123 West Left st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0001", "cf3", "buck,john", "", 4))); - assertEquals(nv("90 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); - assertEquals(nv("1124 East Right st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); - assertEquals(nv("09 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); + assertEquals(nv("1123 West Left st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0001", "cf3", "buck,john", "", 4))); + assertEquals(nv("90 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); + assertEquals(nv("1124 East Right st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); + assertEquals(nv("09 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // scan no loc groups r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, ncfs("saint", "dogooder"), true); + trf.iter.seek(r, ncfs("saint", "dogooder"), true); assertEquals(0, trf.reader.getNumLocalityGroupsSeeked()); - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); // scan a subset of second locality group r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, ncfs("cf4"), true); + trf.iter.seek(r, ncfs("cf4"), true); assertEquals(1, trf.reader.getNumLocalityGroupsSeeked()); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); - assertEquals(nv("09 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); + assertEquals(nv("09 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // scan a subset of second locality group r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, ncfs("cf3"), true); + trf.iter.seek(r, ncfs("cf3"), true); assertEquals(1, trf.reader.getNumLocalityGroupsSeeked()); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0001", "cf3", "buck,john", "", 4))); - assertEquals(nv("90 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0001", "cf3", "buck,john", "", 4))); + assertEquals(nv("90 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // scan subset of first loc group r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, ncfs("cf1"), true); + trf.iter.seek(r, ncfs("cf1"), true); assertEquals(1, trf.reader.getNumLocalityGroupsSeeked()); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); - assertEquals(nv("1123 West Left st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); + assertEquals(nv("1123 West Left st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // scan subset of first loc group r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, ncfs("cf2"), true); + trf.iter.seek(r, ncfs("cf2"), true); assertEquals(1, trf.reader.getNumLocalityGroupsSeeked()); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); - assertEquals(nv("1124 East Right st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); + assertEquals(nv("1124 East Right st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); // scan subset of all loc groups r = new Range(nk("0000", "cf1", "doe,john", "", 4), true, nk("0003", "cf4", "buck,jane", "", 5), true); - trf.reader.seek(r, ncfs("cf1", "cf4"), true); + trf.iter.seek(r, ncfs("cf1", "cf4"), true); assertEquals(2, trf.reader.getNumLocalityGroupsSeeked()); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); - assertEquals(nv("1123 West Left st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); - assertEquals(nv("09 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); + assertEquals(nv("1123 West Left st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); + assertEquals(nv("09 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); trf.closeReader(); @@ -823,8 +825,8 @@ public class RFileTest extends TestCase trf.writer.close(); trf.openReader(); - trf.reader.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); + assertFalse(trf.iter.hasTop()); trf.closeReader(); @@ -840,16 +842,16 @@ public class RFileTest extends TestCase trf.writer.close(); trf.openReader(); - trf.reader.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); - assertEquals(nv("1123 West Left st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); - assertEquals(nv("1124 East Right st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); + assertEquals(nv("1123 West Left st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); + assertEquals(nv("1124 East Right st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); trf.closeReader(); @@ -865,16 +867,16 @@ public class RFileTest extends TestCase trf.writer.close(); trf.openReader(); - trf.reader.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0001", "cf3", "buck,john", "", 4))); - assertEquals(nv("90 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); - assertEquals(nv("09 Slum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0001", "cf3", "buck,john", "", 4))); + assertEquals(nv("90 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0003", "cf4", "buck,jane", "", 5))); + assertEquals(nv("09 Slum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); trf.closeReader(); @@ -890,16 +892,16 @@ public class RFileTest extends TestCase trf.writer.close(); trf.openReader(); - trf.reader.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0007", "good citizen", "q,john", "", 4))); - assertEquals(nv("70 Apple st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0008", "model citizen", "q,jane", "", 5))); - assertEquals(nv("81 Plum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0007", "good citizen", "q,john", "", 4))); + assertEquals(nv("70 Apple st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0008", "model citizen", "q,jane", "", 5))); + assertEquals(nv("81 Plum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); trf.closeReader(); @@ -917,24 +919,24 @@ public class RFileTest extends TestCase trf.writer.close(); trf.openReader(); - trf.reader.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); - assertEquals(nv("1123 West Left st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); - assertEquals(nv("1124 East Right st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0007", "good citizen", "q,john", "", 4))); - assertEquals(nv("70 Apple st"), trf.reader.getTopValue()); - trf.reader.next(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0008", "model citizen", "q,jane", "", 5))); - assertEquals(nv("81 Plum st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); + assertEquals(nv("1123 West Left st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); + assertEquals(nv("1124 East Right st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0007", "good citizen", "q,john", "", 4))); + assertEquals(nv("70 Apple st"), trf.iter.getTopValue()); + trf.iter.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0008", "model citizen", "q,jane", "", 5))); + assertEquals(nv("81 Plum st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); trf.closeReader(); } @@ -973,30 +975,30 @@ public class RFileTest extends TestCase // test a merged read of all column families trf.openReader(); - trf.reader.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); + trf.iter.seek(new Range(new Text(""), null), EMPTY_COL_FAMS, false); assertEquals(3, trf.reader.getNumLocalityGroupsSeeked()); for (int i = 0; i < 1024; i++) { - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("i", i), (i % 10) + "mod10", "", "", i + 2), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("i", i), (i % 10) + "mod10", "", "", i + 2), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); } - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); // try reading each of the 10 column families separately for (int m = 0; m < 10; m++) { - trf.reader.seek(new Range(new Key(), true, null, true), ncfs(m + "mod10"), true); + trf.iter.seek(new Range(new Key(), true, null, true), ncfs(m + "mod10"), true); assertEquals(1, trf.reader.getNumLocalityGroupsSeeked()); for (int i = m; i < 1024; i += 10) { - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("i", i), (i % 10) + "mod10", "", "", i + 2), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("i", i), (i % 10) + "mod10", "", "", i + 2), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); } - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); // test excluding an individual column family - trf.reader.seek(new Range(new Key(), true, null, true), ncfs(m + "mod10"), false); + trf.iter.seek(new Range(new Key(), true, null, true), ncfs(m + "mod10"), false); if (m == 3) assertEquals(2, trf.reader.getNumLocalityGroupsSeeked()); else @@ -1006,29 +1008,29 @@ public class RFileTest extends TestCase if (i % 10 == m) continue; - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("i", i), (i % 10) + "mod10", "", "", i + 2), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("i", i), (i % 10) + "mod10", "", "", i + 2), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); } - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); } // test Rfile deepcopy - SortedKeyValueIterator reader2 = trf.reader.deepCopy(null); + SortedKeyValueIterator reader2 = trf.iter.deepCopy(null); // try reading from cloned reader at the same time as parent reader for (int m = 0; m < 9; m++) { - trf.reader.seek(new Range(new Key(), true, null, true), ncfs(m + "mod10"), true); + trf.iter.seek(new Range(new Key(), true, null, true), ncfs(m + "mod10"), true); assertEquals(1, trf.reader.getNumLocalityGroupsSeeked()); reader2.seek(new Range(new Key(), true, null, true), ncfs((m + 1) + "mod10"), true); // assertEquals(1, reader2.getNumLocalityGroupsSeeked()); for (int i = m; i < 1024; i += 10) { // System.out.println(m+","+i); - assertTrue(trf.reader.hasTop()); - assertEquals(nk(nf("i", i), (i % 10) + "mod10", "", "", i + 2), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk(nf("i", i), (i % 10) + "mod10", "", "", i + 2), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); if (i + 1 < 1024) { assertTrue(reader2.hasTop()); assertEquals(nk(nf("i", (i + 1)), ((i + 1) % 10) + "mod10", "", "", i + 3), reader2.getTopKey()); @@ -1036,7 +1038,7 @@ public class RFileTest extends TestCase reader2.next(); } } - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); assertFalse(reader2.hasTop()); } @@ -1065,12 +1067,12 @@ public class RFileTest extends TestCase trf.openReader(); - trf.reader.seek(new Range(), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertEquals(nk("0007", "a", "cq1", "", 4), trf.reader.getTopKey()); - assertEquals(nv("1"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(), EMPTY_COL_FAMS, false); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("0007", "a", "cq1", "", 4), trf.iter.getTopKey()); + assertEquals(nv("1"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); } @@ -1106,12 +1108,12 @@ public class RFileTest extends TestCase trf.openReader(); - trf.reader.seek(new Range(), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertEquals(nk("0007", "a", "cq1", "", 4), trf.reader.getTopKey()); - assertEquals(nv("1"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(), EMPTY_COL_FAMS, false); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("0007", "a", "cq1", "", 4), trf.iter.getTopKey()); + assertEquals(nv("1"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); } @@ -1189,59 +1191,59 @@ public class RFileTest extends TestCase assert (count > 4); - trf.reader.seek(new Range(nk("r0000", "cf1", "cq1", "", 1), true, nk("r0001", "cf1", "cq1", "", 1), false), EMPTY_COL_FAMS, false); + trf.iter.seek(new Range(nk("r0000", "cf1", "cq1", "", 1), true, nk("r0001", "cf1", "cq1", "", 1), false), EMPTY_COL_FAMS, false); for (int i = 0; i < 2048; i++) { - assertTrue(trf.reader.hasTop()); - assertEquals(nk("r0000", "cf1", "cq1", "", 1), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("r0000", "cf1", "cq1", "", 1), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); } - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); - trf.reader.seek(new Range(nk("r0000", "cf1", "cq1", "", 1), false, nk("r0001", "cf1", "cq1", "", 1), true), EMPTY_COL_FAMS, false); + trf.iter.seek(new Range(nk("r0000", "cf1", "cq1", "", 1), false, nk("r0001", "cf1", "cq1", "", 1), true), EMPTY_COL_FAMS, false); for (int i = 2048; i < 4096; i++) { - assertTrue(trf.reader.hasTop()); - assertEquals(nk("r0001", "cf1", "cq1", "", 1), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("r0001", "cf1", "cq1", "", 1), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); } - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); - trf.reader.seek(new Range(nk("r0001", "cf1", "cq1", "", 1), true, nk("r0001", "cf1", "cq1", "", 1), true), EMPTY_COL_FAMS, false); + trf.iter.seek(new Range(nk("r0001", "cf1", "cq1", "", 1), true, nk("r0001", "cf1", "cq1", "", 1), true), EMPTY_COL_FAMS, false); for (int i = 2048; i < 4096; i++) { - assertTrue(trf.reader.hasTop()); - assertEquals(nk("r0001", "cf1", "cq1", "", 1), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("r0001", "cf1", "cq1", "", 1), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); } - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); - trf.reader.seek(new Range(nk("r0002", "cf1", "cq1", "", 1), true, nk("r0002", "cf1", "cq1", "", 1), true), EMPTY_COL_FAMS, false); - assertFalse(trf.reader.hasTop()); + trf.iter.seek(new Range(nk("r0002", "cf1", "cq1", "", 1), true, nk("r0002", "cf1", "cq1", "", 1), true), EMPTY_COL_FAMS, false); + assertFalse(trf.iter.hasTop()); - trf.reader.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false); + trf.iter.seek(new Range((Key) null, null), EMPTY_COL_FAMS, false); for (int i = 0; i < 2048; i++) { - assertTrue(trf.reader.hasTop()); - assertEquals(nk("r0000", "cf1", "cq1", "", 1), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("r0000", "cf1", "cq1", "", 1), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); } for (int i = 2048; i < 4096; i++) { - assertTrue(trf.reader.hasTop()); - assertEquals(nk("r0001", "cf1", "cq1", "", 1), trf.reader.getTopKey()); - assertEquals(nv("" + i), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertEquals(nk("r0001", "cf1", "cq1", "", 1), trf.iter.getTopKey()); + assertEquals(nv("" + i), trf.iter.getTopValue()); + trf.iter.next(); } - assertFalse(trf.reader.hasTop()); + assertFalse(trf.iter.hasTop()); trf.closeReader(); } @@ -1265,29 +1267,30 @@ public class RFileTest extends TestCase allCf.add(new ArrayByteSequence(cf)); } - private void t18Verify(Set cfs, Reader reader, HashSet allCf, int eialg, int eealg) throws IOException { + private void t18Verify(Set cfs, SortedKeyValueIterator iter, Reader reader, HashSet allCf, int eialg, int eealg) + throws IOException { HashSet colFamsSeen = new HashSet(); - reader.seek(new Range(), cfs, true); + iter.seek(new Range(), cfs, true); assertEquals(eialg, reader.getNumLocalityGroupsSeeked()); - while (reader.hasTop()) { - colFamsSeen.add(reader.getTopKey().getColumnFamilyData()); - reader.next(); + while (iter.hasTop()) { + colFamsSeen.add(iter.getTopKey().getColumnFamilyData()); + iter.next(); } HashSet expected = new HashSet(allCf); expected.retainAll(cfs); assertEquals(expected, colFamsSeen); - reader.seek(new Range(), cfs, false); + iter.seek(new Range(), cfs, false); assertEquals(eealg, reader.getNumLocalityGroupsSeeked()); colFamsSeen.clear(); - while (reader.hasTop()) { - colFamsSeen.add(reader.getTopKey().getColumnFamilyData()); - reader.next(); + while (iter.hasTop()) { + colFamsSeen.add(iter.getTopKey().getColumnFamilyData()); + iter.next(); } HashSet nonExcluded = new HashSet(allCf); @@ -1326,22 +1329,22 @@ public class RFileTest extends TestCase trf.openReader(); - t18Verify(t18ncfs(0), trf.reader, allCf, 1, 3); + t18Verify(t18ncfs(0), trf.iter, trf.reader, allCf, 1, 3); for (int i = 1; i < 10; i++) - t18Verify(t18ncfs(i), trf.reader, allCf, 1, 4); + t18Verify(t18ncfs(i), trf.iter, trf.reader, allCf, 1, 4); - t18Verify(t18ncfs(max + 1), trf.reader, allCf, 1, 4); + t18Verify(t18ncfs(max + 1), trf.iter, trf.reader, allCf, 1, 4); - t18Verify(t18ncfs(1, 2, 3, 4), trf.reader, allCf, 2, 3); - t18Verify(t18ncfs(1, 2, 3, 4, 5), trf.reader, allCf, 2, 2); + t18Verify(t18ncfs(1, 2, 3, 4), trf.iter, trf.reader, allCf, 2, 3); + t18Verify(t18ncfs(1, 2, 3, 4, 5), trf.iter, trf.reader, allCf, 2, 2); - t18Verify(t18ncfs(0, 1, 2, 3, 4), trf.reader, allCf, 3, 2); - t18Verify(t18ncfs(0, 1, 2, 3, 4, 5), trf.reader, allCf, 3, 1); - t18Verify(t18ncfs(0, 1, 2, 3, 4, 5, 6), trf.reader, allCf, 4, 1); + t18Verify(t18ncfs(0, 1, 2, 3, 4), trf.iter, trf.reader, allCf, 3, 2); + t18Verify(t18ncfs(0, 1, 2, 3, 4, 5), trf.iter, trf.reader, allCf, 3, 1); + t18Verify(t18ncfs(0, 1, 2, 3, 4, 5, 6), trf.iter, trf.reader, allCf, 4, 1); - t18Verify(t18ncfs(0, 1), trf.reader, allCf, 2, 3); - t18Verify(t18ncfs(2, 3), trf.reader, allCf, 2, 4); - t18Verify(t18ncfs(5, 6), trf.reader, allCf, 2, 4); + t18Verify(t18ncfs(0, 1), trf.iter, trf.reader, allCf, 2, 3); + t18Verify(t18ncfs(2, 3), trf.iter, trf.reader, allCf, 2, 4); + t18Verify(t18ncfs(5, 6), trf.iter, trf.reader, allCf, 2, 4); trf.closeReader(); } @@ -1372,12 +1375,12 @@ public class RFileTest extends TestCase trf.openReader(); - trf.reader.seek(new Range(), EMPTY_COL_FAMS, false); + trf.iter.seek(new Range(), EMPTY_COL_FAMS, false); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); - assertEquals(nv("1123 West Left st"), trf.reader.getTopValue()); - trf.reader.next(); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0000", "cf1", "doe,john", "", 4))); + assertEquals(nv("1123 West Left st"), trf.iter.getTopValue()); + trf.iter.next(); DataInputStream in = trf.reader.getMetaStore("count"); @@ -1389,11 +1392,11 @@ public class RFileTest extends TestCase in.close(); - assertTrue(trf.reader.hasTop()); - assertTrue(trf.reader.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); - assertEquals(nv("1124 East Right st"), trf.reader.getTopValue()); - trf.reader.next(); - assertFalse(trf.reader.hasTop()); + assertTrue(trf.iter.hasTop()); + assertTrue(trf.iter.getTopKey().equals(nk("0002", "cf2", "doe,jane", "", 5))); + assertEquals(nv("1124 East Right st"), trf.iter.getTopValue()); + trf.iter.next(); + assertFalse(trf.iter.hasTop()); trf.closeReader(); } @@ -1418,38 +1421,40 @@ public class RFileTest extends TestCase Reader reader = new RFile.Reader(_cbr); checkIndex(reader); + ColumnFamilySkippingIterator iter = new ColumnFamilySkippingIterator(reader); + for (int start : new int[] {0, 10, 100, 998}) { for (int cf = 1; cf <= 4; cf++) { if (start == 0) - reader.seek(new Range(), ncfs(nf("cf_", cf)), true); + iter.seek(new Range(), ncfs(nf("cf_", cf)), true); else - reader.seek(new Range(nf("r_", start), null), ncfs(nf("cf_", cf)), true); + iter.seek(new Range(nf("r_", start), null), ncfs(nf("cf_", cf)), true); for (int i = start; i < 1000; i++) { - assertTrue(reader.hasTop()); - assertEquals(nk(nf("r_", i), nf("cf_", cf), nf("cq_", 0), "", 1000 - i), reader.getTopKey()); - assertEquals(nv(i + ""), reader.getTopValue()); - reader.next(); + assertTrue(iter.hasTop()); + assertEquals(nk(nf("r_", i), nf("cf_", cf), nf("cq_", 0), "", 1000 - i), iter.getTopKey()); + assertEquals(nv(i + ""), iter.getTopValue()); + iter.next(); } - assertFalse(reader.hasTop()); + assertFalse(iter.hasTop()); } if (start == 0) - reader.seek(new Range(), ncfs(), false); + iter.seek(new Range(), ncfs(), false); else - reader.seek(new Range(nf("r_", start), null), ncfs(), false); + iter.seek(new Range(nf("r_", start), null), ncfs(), false); for (int i = start; i < 1000; i++) { for (int cf = 1; cf <= 4; cf++) { - assertTrue(reader.hasTop()); - assertEquals(nk(nf("r_", i), nf("cf_", cf), nf("cq_", 0), "", 1000 - i), reader.getTopKey()); - assertEquals(nv(i + ""), reader.getTopValue()); - reader.next(); + assertTrue(iter.hasTop()); + assertEquals(nk(nf("r_", i), nf("cf_", cf), nf("cq_", 0), "", 1000 - i), iter.getTopKey()); + assertEquals(nv(i + ""), iter.getTopValue()); + iter.next(); } } - assertFalse(reader.hasTop()); + assertFalse(iter.hasTop()); } reader.close(); Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Compactor.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Compactor.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Compactor.java (original) +++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Compactor.java Fri Apr 13 17:21:07 2012 @@ -36,8 +36,9 @@ import org.apache.accumulo.core.file.Fil import org.apache.accumulo.core.file.FileSKVIterator; import org.apache.accumulo.core.file.FileSKVWriter; import org.apache.accumulo.core.iterators.IteratorUtil; -import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; +import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator; import org.apache.accumulo.core.iterators.system.CountingIterator; import org.apache.accumulo.core.iterators.system.DeletingIterator; import org.apache.accumulo.core.iterators.system.MultiIterator; @@ -247,7 +248,9 @@ public class Compactor implements Callab CountingIterator citr = new CountingIterator(new MultiIterator(iters, extent.toDataRange())); DeletingIterator delIter = new DeletingIterator(citr, propogateDeletes); + ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter); + // if(env.getIteratorScope() ) TabletIteratorEnvironment iterEnv; @@ -258,7 +261,7 @@ public class Compactor implements Callab else throw new IllegalArgumentException(); - SortedKeyValueIterator itr = iterEnv.getTopLevelIterator(IteratorUtil.loadIterators(env.getIteratorScope(), delIter, extent, acuTableConf, + SortedKeyValueIterator itr = iterEnv.getTopLevelIterator(IteratorUtil.loadIterators(env.getIteratorScope(), cfsi, extent, acuTableConf, iterEnv)); itr.seek(extent.toDataRange(), columnFamilies, inclusive); Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java (original) +++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/InMemoryMap.java Fri Apr 13 17:21:07 2012 @@ -48,7 +48,6 @@ import org.apache.accumulo.core.iterator import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.iterators.SortedMapIterator; import org.apache.accumulo.core.iterators.WrappingIterator; -import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator; import org.apache.accumulo.core.iterators.system.InterruptibleIterator; import org.apache.accumulo.core.iterators.system.SourceSwitchingIterator; import org.apache.accumulo.core.iterators.system.SourceSwitchingIterator.DataSource; @@ -452,7 +451,7 @@ public class InMemoryMap { int mc = mutationCount.get(); MemoryDataSource mds = new MemoryDataSource(); SourceSwitchingIterator ssi = new SourceSwitchingIterator(new MemoryDataSource()); - MemoryIterator mi = new MemoryIterator(new ColumnFamilySkippingIterator(new PartialMutationSkippingIterator(ssi, mc))); + MemoryIterator mi = new MemoryIterator(new PartialMutationSkippingIterator(ssi, mc)); mi.setSSI(ssi); mi.setMDS(mds); activeIters.add(mi); @@ -465,7 +464,7 @@ public class InMemoryMap { throw new IllegalStateException("Memory map in unexpected state : nextMutationCount = " + nextMutationCount.get() + " mutationCount = " + mutationCount.get()); - return new ColumnFamilySkippingIterator(map.skvIterator()); + return map.skvIterator(); } private boolean deleted = false; Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/NativeMap.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/NativeMap.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/NativeMap.java (original) +++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/NativeMap.java Fri Apr 13 17:21:07 2012 @@ -24,8 +24,8 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.NoSuchElementException; import java.util.Map.Entry; +import java.util.NoSuchElementException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReadWriteLock; @@ -669,10 +669,6 @@ public class NativeMap implements Iterab @Override public void seek(Range range, Collection columnFamilies, boolean inclusive) throws IOException { - if (columnFamilies.size() != 0 || inclusive) { - throw new IllegalArgumentException("I do not know how to filter column families"); - } - if (interruptFlag != null && interruptFlag.get()) throw new IterationInterruptedException(); Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java (original) +++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/tabletserver/Tablet.java Fri Apr 13 17:21:07 2012 @@ -74,6 +74,7 @@ import org.apache.accumulo.core.iterator import org.apache.accumulo.core.iterators.IteratorUtil; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator; import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter; import org.apache.accumulo.core.iterators.system.DeletingIterator; import org.apache.accumulo.core.iterators.system.InterruptibleIterator; @@ -2076,7 +2077,9 @@ public class Tablet { DeletingIterator delIter = new DeletingIterator(multiIter, false); - ColumnQualifierFilter colFilter = new ColumnQualifierFilter(delIter, options.columnSet); + ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter); + + ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, options.columnSet); VisibilityFilter visFilter = new VisibilityFilter(colFilter, options.authorizations, options.defaultLabels); Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/SparseColumnFamilyTest.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/SparseColumnFamilyTest.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/SparseColumnFamilyTest.java (original) +++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/functional/SparseColumnFamilyTest.java Fri Apr 13 17:21:07 2012 @@ -86,6 +86,9 @@ public class SparseColumnFamilyTest exte Iterator> iter = scanner.iterator(); if (iter.hasNext()) { Entry entry = iter.next(); + if (!"001".equals(entry.getKey().getColumnFamilyData().toString())) { + throw new Exception(); + } } long t2 = System.currentTimeMillis(); Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java (original) +++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/test/performance/scan/CollectTabletStats.java Fri Apr 13 17:21:07 2012 @@ -433,11 +433,12 @@ public class CollectTabletStats { List> iters = new ArrayList>(mapfiles.size() + 1); iters.addAll(mapfiles); - iters.add(new ColumnFamilySkippingIterator(smi)); + iters.add(smi); MultiIterator multiIter = new MultiIterator(iters, ke); DeletingIterator delIter = new DeletingIterator(multiIter, false); - ColumnQualifierFilter colFilter = new ColumnQualifierFilter(delIter, columnSet); + ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter); + ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, columnSet); VisibilityFilter visFilter = new VisibilityFilter(colFilter, authorizations, defaultLabels); if (useTableIterators) Modified: accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java (original) +++ accumulo/branches/1.4/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java Fri Apr 13 17:21:07 2012 @@ -38,6 +38,7 @@ import org.apache.accumulo.core.data.Val import org.apache.accumulo.core.file.FileOperations; import org.apache.accumulo.core.file.FileSKVIterator; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator; import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter; import org.apache.accumulo.core.iterators.system.DeletingIterator; import org.apache.accumulo.core.iterators.system.MultiIterator; @@ -72,7 +73,8 @@ public class OfflineMetadataScanner exte throws IOException { MultiIterator multiIterator = new MultiIterator(readers, false); DeletingIterator delIter = new DeletingIterator(multiIterator, false); - ColumnQualifierFilter colFilter = new ColumnQualifierFilter(delIter, columns); + ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(delIter); + ColumnQualifierFilter colFilter = new ColumnQualifierFilter(cfsi, columns); VisibilityFilter visFilter = new VisibilityFilter(colFilter, Constants.NO_AUTHS, new byte[0]); visFilter.seek(r, LocalityGroupUtil.EMPTY_CF_SET, false); Modified: accumulo/branches/1.4/src/server/src/test/java/org/apache/accumulo/server/tabletserver/InMemoryMapTest.java URL: http://svn.apache.org/viewvc/accumulo/branches/1.4/src/server/src/test/java/org/apache/accumulo/server/tabletserver/InMemoryMapTest.java?rev=1325853&r1=1325852&r2=1325853&view=diff ============================================================================== --- accumulo/branches/1.4/src/server/src/test/java/org/apache/accumulo/server/tabletserver/InMemoryMapTest.java (original) +++ accumulo/branches/1.4/src/server/src/test/java/org/apache/accumulo/server/tabletserver/InMemoryMapTest.java Fri Apr 13 17:21:07 2012 @@ -29,10 +29,10 @@ import org.apache.accumulo.core.data.Mut import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; +import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator; import org.apache.accumulo.core.util.LocalityGroupUtil; import org.apache.accumulo.server.client.HdfsZooInstance; import org.apache.accumulo.server.conf.ZooConfiguration; -import org.apache.accumulo.server.tabletserver.InMemoryMap; import org.apache.accumulo.server.tabletserver.InMemoryMap.MemoryIterator; import org.apache.hadoop.io.Text; import org.apache.log4j.Level; @@ -245,6 +245,7 @@ public class InMemoryMapTest extends Tes } MemoryIterator ski1 = imm.skvIterator(); + ColumnFamilySkippingIterator cfsi = new ColumnFamilySkippingIterator(ski1); imm.delete(0); @@ -252,9 +253,9 @@ public class InMemoryMapTest extends Tes columns.add(new ArrayByteSequence("bar")); // this seek resulted in an infinite loop before a bug was fixed - ski1.seek(new Range("r1"), columns, true); + cfsi.seek(new Range("r1"), columns, true); - assertFalse(ski1.hasTop()); + assertFalse(cfsi.hasTop()); ski1.close(); }