Return-Path: X-Original-To: apmail-incubator-accumulo-commits-archive@minotaur.apache.org Delivered-To: apmail-incubator-accumulo-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 12B617C75 for ; Thu, 17 Nov 2011 22:32:12 +0000 (UTC) Received: (qmail 20613 invoked by uid 500); 17 Nov 2011 22:32:12 -0000 Delivered-To: apmail-incubator-accumulo-commits-archive@incubator.apache.org Received: (qmail 20593 invoked by uid 500); 17 Nov 2011 22:32:12 -0000 Mailing-List: contact accumulo-commits-help@incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: accumulo-dev@incubator.apache.org Delivered-To: mailing list accumulo-commits@incubator.apache.org Received: (qmail 20586 invoked by uid 99); 17 Nov 2011 22:32:11 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 17 Nov 2011 22:32:11 +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; Thu, 17 Nov 2011 22:32:01 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id E1A6823889D7; Thu, 17 Nov 2011 22:31:38 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1203397 - in /incubator/accumulo/trunk: ./ src/core/src/main/java/org/apache/accumulo/core/client/admin/ src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/ src/core/src/main/java/org/apache/accumulo/core/client/mock/ src/cor... Date: Thu, 17 Nov 2011 22:31:37 -0000 To: accumulo-commits@incubator.apache.org From: billie@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20111117223138.E1A6823889D7@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: billie Date: Thu Nov 17 22:31:36 2011 New Revision: 1203397 URL: http://svn.apache.org/viewvc?rev=1203397&view=rev Log: ACCUMULO-155 merged to trunk Added: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java - copied unchanged from r1203389, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java - copied unchanged from r1203389, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java - copied unchanged from r1203389, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java - copied unchanged from r1203389, incubator/accumulo/branches/1.4/src/core/src/main/java/org/apache/accumulo/core/iterators/user/VersioningIterator.java Modified: incubator/accumulo/trunk/ (props changed) incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java Propchange: incubator/accumulo/trunk/ ------------------------------------------------------------------------------ --- svn:mergeinfo (original) +++ svn:mergeinfo Thu Nov 17 22:31:36 2011 @@ -1,2 +1,2 @@ /incubator/accumulo/branches/1.3:1190280,1190413,1190420,1190427,1190500,1195622,1195625,1195629,1195635,1196044,1196054,1196057,1196071-1196072,1196106,1197066,1198935,1199383 -/incubator/accumulo/branches/1.4:1201902-1203347 +/incubator/accumulo/branches/1.4:1201902-1203389 Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java Thu Nov 17 22:31:36 2011 @@ -59,7 +59,6 @@ import org.apache.accumulo.core.data.Byt import org.apache.accumulo.core.data.KeyExtent; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.iterators.IteratorUtil; -import org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig; import org.apache.accumulo.core.master.state.tables.TableState; import org.apache.accumulo.core.master.thrift.MasterClientService; import org.apache.accumulo.core.master.thrift.TableOperation; @@ -182,8 +181,7 @@ public class TableOperationsImpl extends Map opts; if (limitVersion) { - List emptyArgs = Collections.emptyList(); - opts = IteratorUtil.generateInitialTableProperties(emptyArgs); + opts = IteratorUtil.generateInitialTableProperties(); } else opts = Collections.emptyMap(); @@ -1039,13 +1037,14 @@ public class TableOperationsImpl extends * @deprecated since 1.4 {@link #attachIterator(String, IteratorSetting)} */ @Override - public void addAggregators(String tableName, List aggregators) throws AccumuloSecurityException, TableNotFoundException, + public void addAggregators(String tableName, List aggregators) + throws AccumuloSecurityException, TableNotFoundException, AccumuloException { ArgumentChecker.notNull(tableName, aggregators); MasterClientService.Iface client = null; try { client = MasterClient.getConnection(instance); - for (Entry entry : IteratorUtil.generateInitialTableProperties(aggregators).entrySet()) { + for (Entry entry : IteratorUtil.generateAggTableProperties(aggregators).entrySet()) { client.setTableProperty(null, credentials, tableName, entry.getKey(), entry.getValue()); } } catch (ThriftSecurityException e) { Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java Thu Nov 17 22:31:36 2011 @@ -1,12 +1,18 @@ /* - * 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. + * 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.accumulo.core.client.mapreduce; @@ -55,7 +61,7 @@ import org.apache.accumulo.core.data.Par 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.VersioningIterator; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.TablePermission; import org.apache.accumulo.core.security.thrift.AuthInfo; Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java Thu Nov 17 22:31:36 2011 @@ -16,7 +16,6 @@ */ package org.apache.accumulo.core.client.mock; -import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.List; @@ -25,6 +24,7 @@ import java.util.Map.Entry; import java.util.SortedMap; import java.util.concurrent.ConcurrentSkipListMap; +import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; @@ -33,7 +33,6 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.iterators.IteratorUtil; -import org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig; import org.apache.accumulo.core.security.TablePermission; public class MockTable { @@ -88,24 +87,7 @@ public class MockTable { MockTable(boolean useVersions, TimeType timeType) { this.timeType = timeType; - List aggs = Collections.emptyList(); - settings = IteratorUtil.generateInitialTableProperties(aggs); - for (Entry entry : AccumuloConfiguration.getDefaultConfiguration()) { - String key = entry.getKey(); - if (key.startsWith(Property.TABLE_PREFIX.getKey())) - settings.put(key, entry.getValue()); - } - } - - /** - * @deprecated since 1.4, use {@link #MockTable(boolean, TimeType)} - * @see {@link #addAggregators(List)} - * @param aggregators - * @param timeType - */ - MockTable(List aggregators, TimeType timeType) { - this.timeType = timeType; - settings = IteratorUtil.generateInitialTableProperties(aggregators); + settings = IteratorUtil.generateInitialTableProperties(); for (Entry entry : AccumuloConfiguration.getDefaultConfiguration()) { String key = entry.getKey(); if (key.startsWith(Property.TABLE_PREFIX.getKey())) @@ -131,8 +113,11 @@ public class MockTable { } } - public void addAggregators(List aggregators) { - for (Entry entry : IteratorUtil.generateInitialTableProperties(aggregators).entrySet()) { + /** + * @deprecated since 1.4 {@link #attachIterator(String, IteratorSetting)} + */ + public void addAggregators(List aggregators) { + for (Entry entry : IteratorUtil.generateAggTableProperties(aggregators).entrySet()) { String key = entry.getKey(); if (key.startsWith(Property.TABLE_PREFIX.getKey())) settings.put(key, entry.getValue()); Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/FamilyIntersectingIterator.java Thu Nov 17 22:31:36 2011 @@ -28,6 +28,7 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.PartialKey; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; +import org.apache.accumulo.core.iterators.user.IntersectingIterator; import org.apache.hadoop.io.Text; public class FamilyIntersectingIterator extends IntersectingIterator { Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/GrepIterator.java Thu Nov 17 22:31:36 2011 @@ -16,88 +16,12 @@ */ package org.apache.accumulo.core.iterators; -import java.io.IOException; -import java.util.Map; - -import org.apache.accumulo.core.client.IteratorSetting; -import org.apache.accumulo.core.data.ByteSequence; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.Value; - -public class GrepIterator extends SkippingIterator { - - private byte term[]; - - @Override - protected void consume() throws IOException { - while (getSource().hasTop()) { - Key k = getSource().getTopKey(); - Value v = getSource().getTopValue(); - - if (match(v.get()) || match(k.getRowData()) || match(k.getColumnFamilyData()) || match(k.getColumnQualifierData())) { - break; - } - - getSource().next(); - } - } - - private boolean match(ByteSequence bs) { - return indexOf(bs.getBackingArray(), bs.offset(), bs.length(), term) >= 0; - } - - private boolean match(byte[] ba) { - return indexOf(ba, 0, ba.length, term) >= 0; - } - - // copied code below from java string and modified - - private static int indexOf(byte[] source, int sourceOffset, int sourceCount, byte[] target) { - byte first = target[0]; - int targetCount = target.length; - int max = sourceOffset + (sourceCount - targetCount); - - for (int i = sourceOffset; i <= max; i++) { - /* Look for first character. */ - if (source[i] != first) { - while (++i <= max && source[i] != first) - continue; - } - - /* Found first character, now look at the rest of v2 */ - if (i <= max) { - int j = i + 1; - int end = j + targetCount - 1; - for (int k = 1; j < end && source[j] == target[k]; j++, k++) - continue; - - if (j == end) { - /* Found whole string. */ - return i - sourceOffset; - } - } - } - return -1; - } - - @Override - public SortedKeyValueIterator deepCopy(IteratorEnvironment env) { - throw new UnsupportedOperationException(); - } - - @Override - public void init(SortedKeyValueIterator source, Map options, IteratorEnvironment env) throws IOException { - super.init(source, options, env); - term = options.get("term").getBytes(); - } +/** + * This class remains here for backwards compatibility. + * + * @deprecated since 1.4 + * @see org.apache.accumulo.core.iterators.user.GrepIterator + */ +public class GrepIterator extends org.apache.accumulo.core.iterators.user.GrepIterator { - /** - * Encode the grep term as an option for a ScanIterator - * - * @param cfg - * @param term - */ - public static void setTerm(IteratorSetting cfg, String term) { - cfg.addOption("term", term); - } } Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IntersectingIterator.java Thu Nov 17 22:31:36 2011 @@ -16,493 +16,12 @@ */ package org.apache.accumulo.core.iterators; -import java.io.IOException; -import java.util.Collection; -import java.util.Map; - -import org.apache.accumulo.core.client.IteratorSetting; -import org.apache.accumulo.core.data.ByteSequence; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.PartialKey; -import org.apache.accumulo.core.data.Range; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.util.TextUtil; -import org.apache.commons.codec.binary.Base64; -import org.apache.hadoop.io.Text; -import org.apache.log4j.Logger; - -public class IntersectingIterator implements SortedKeyValueIterator { - - protected Text nullText = new Text(); - - protected Text getPartition(Key key) { - return key.getRow(); - } - - protected Text getTerm(Key key) { - return key.getColumnFamily(); - } - - protected Text getDocID(Key key) { - return key.getColumnQualifier(); - } - - protected Key buildKey(Text partition, Text term) { - return new Key(partition, (term == null) ? nullText : term); - } - - protected Key buildKey(Text partition, Text term, Text docID) { - return new Key(partition, (term == null) ? nullText : term, docID); - } - - protected Key buildFollowingPartitionKey(Key key) { - return key.followingKey(PartialKey.ROW); - } - - protected static final Logger log = Logger.getLogger(IntersectingIterator.class); - - protected static class TermSource { - public SortedKeyValueIterator iter; - public Text term; - public boolean notFlag; - - public TermSource(TermSource other) { - this.iter = other.iter; - this.term = other.term; - this.notFlag = other.notFlag; - } - - public TermSource(SortedKeyValueIterator iter, Text term) { - this.iter = iter; - this.term = term; - this.notFlag = false; - } - - public TermSource(SortedKeyValueIterator iter, Text term, boolean notFlag) { - this.iter = iter; - this.term = term; - this.notFlag = notFlag; - } - - public String getTermString() { - return (this.term == null) ? new String("Iterator") : this.term.toString(); - } - } - - TermSource[] sources; - int sourcesCount = 0; - - Range overallRange; - - // query-time settings - Text currentPartition = null; - Text currentDocID = new Text(emptyByteArray); - static final byte[] emptyByteArray = new byte[0]; - - Key topKey = null; - Value value = new Value(emptyByteArray); - - protected Collection seekColumnFamilies; - - protected boolean inclusive; - - public IntersectingIterator() {} - - @Override - public SortedKeyValueIterator deepCopy(IteratorEnvironment env) { - return new IntersectingIterator(this, env); - } - - public IntersectingIterator(IntersectingIterator other, IteratorEnvironment env) { - if (other.sources != null) { - sourcesCount = other.sourcesCount; - sources = new TermSource[sourcesCount]; - for (int i = 0; i < sourcesCount; i++) { - sources[i] = new TermSource(other.sources[i].iter.deepCopy(env), other.sources[i].term); - } - } - } - - @Override - public Key getTopKey() { - return topKey; - } - - @Override - public Value getTopValue() { - // we don't really care about values - return value; - } - - @Override - public boolean hasTop() { - return currentPartition != null; - } - - // precondition: currentRow is not null - private boolean seekOneSource(int sourceID) throws IOException { - // find the next key in the appropriate column family that is at or beyond the cursor (currentRow, currentCQ) - // advance the cursor if this source goes beyond it - // return whether we advanced the cursor - - // within this loop progress must be made in one of the following forms: - // - currentRow or currentCQ must be increased - // - the given source must advance its iterator - // this loop will end when any of the following criteria are met - // - the iterator for the given source is pointing to the key (currentRow, columnFamilies[sourceID], currentCQ) - // - the given source is out of data and currentRow is set to null - // - the given source has advanced beyond the endRow and currentRow is set to null - boolean advancedCursor = false; - - if (sources[sourceID].notFlag) { - while (true) { - if (sources[sourceID].iter.hasTop() == false) { - // an empty column that you are negating is a valid condition - break; - } - // check if we're past the end key - int endCompare = -1; - // we should compare the row to the end of the range - if (overallRange.getEndKey() != null) { - endCompare = overallRange.getEndKey().getRow().compareTo(sources[sourceID].iter.getTopKey().getRow()); - if ((!overallRange.isEndKeyInclusive() && endCompare <= 0) || endCompare < 0) { - // an empty column that you are negating is a valid condition - break; - } - } - int partitionCompare = currentPartition.compareTo(getPartition(sources[sourceID].iter.getTopKey())); - // check if this source is already at or beyond currentRow - // if not, then seek to at least the current row - - if (partitionCompare > 0) { - // seek to at least the currentRow - Key seekKey = buildKey(currentPartition, sources[sourceID].term); - sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive); - continue; - } - // check if this source has gone beyond currentRow - // if so, this is a valid condition for negation - if (partitionCompare < 0) { - break; - } - // we have verified that the current source is positioned in currentRow - // now we must make sure we're in the right columnFamily in the current row - // Note: Iterators are auto-magically set to the correct columnFamily - if (sources[sourceID].term != null) { - int termCompare = sources[sourceID].term.compareTo(getTerm(sources[sourceID].iter.getTopKey())); - // check if this source is already on the right columnFamily - // if not, then seek forwards to the right columnFamily - if (termCompare > 0) { - Key seekKey = buildKey(currentPartition, sources[sourceID].term, currentDocID); - sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive); - continue; - } - // check if this source is beyond the right columnFamily - // if so, then this is a valid condition for negating - if (termCompare < 0) { - break; - } - } - - // we have verified that we are in currentRow and the correct column family - // make sure we are at or beyond columnQualifier - Text docID = getDocID(sources[sourceID].iter.getTopKey()); - int docIDCompare = currentDocID.compareTo(docID); - // If we are past the target, this is a valid result - if (docIDCompare < 0) { - break; - } - // if this source is not yet at the currentCQ then advance in this source - if (docIDCompare > 0) { - // seek forwards - Key seekKey = buildKey(currentPartition, sources[sourceID].term, currentDocID); - sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive); - continue; - } - // if we are equal to the target, this is an invalid result. - // Force the entire process to go to the next row. - // We are advancing column 0 because we forced that column to not contain a ! - // when we did the init() - if (docIDCompare == 0) { - sources[0].iter.next(); - advancedCursor = true; - break; - } - } - } else { - while (true) { - if (sources[sourceID].iter.hasTop() == false) { - currentPartition = null; - // setting currentRow to null counts as advancing the cursor - return true; - } - // check if we're past the end key - int endCompare = -1; - // we should compare the row to the end of the range - - if (overallRange.getEndKey() != null) { - endCompare = overallRange.getEndKey().getRow().compareTo(sources[sourceID].iter.getTopKey().getRow()); - if ((!overallRange.isEndKeyInclusive() && endCompare <= 0) || endCompare < 0) { - currentPartition = null; - // setting currentRow to null counts as advancing the cursor - return true; - } - } - int partitionCompare = currentPartition.compareTo(getPartition(sources[sourceID].iter.getTopKey())); - // check if this source is already at or beyond currentRow - // if not, then seek to at least the current row - if (partitionCompare > 0) { - // seek to at least the currentRow - Key seekKey = buildKey(currentPartition, sources[sourceID].term); - sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive); - continue; - } - // check if this source has gone beyond currentRow - // if so, advance currentRow - if (partitionCompare < 0) { - currentPartition.set(getPartition(sources[sourceID].iter.getTopKey())); - currentDocID.set(emptyByteArray); - advancedCursor = true; - continue; - } - // we have verified that the current source is positioned in currentRow - // now we must make sure we're in the right columnFamily in the current row - // Note: Iterators are auto-magically set to the correct columnFamily - - if (sources[sourceID].term != null) { - int termCompare = sources[sourceID].term.compareTo(getTerm(sources[sourceID].iter.getTopKey())); - // check if this source is already on the right columnFamily - // if not, then seek forwards to the right columnFamily - if (termCompare > 0) { - Key seekKey = buildKey(currentPartition, sources[sourceID].term, currentDocID); - sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive); - continue; - } - // check if this source is beyond the right columnFamily - // if so, then seek to the next row - if (termCompare < 0) { - // we're out of entries in the current row, so seek to the next one - // byte[] currentRowBytes = currentRow.getBytes(); - // byte[] nextRow = new byte[currentRowBytes.length + 1]; - // System.arraycopy(currentRowBytes, 0, nextRow, 0, currentRowBytes.length); - // nextRow[currentRowBytes.length] = (byte)0; - // // we should reuse text objects here - // sources[sourceID].seek(new Key(new Text(nextRow),columnFamilies[sourceID])); - if (endCompare == 0) { - // we're done - currentPartition = null; - // setting currentRow to null counts as advancing the cursor - return true; - } - Key seekKey = buildFollowingPartitionKey(sources[sourceID].iter.getTopKey()); - sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive); - continue; - } - } - // we have verified that we are in currentRow and the correct column family - // make sure we are at or beyond columnQualifier - Text docID = getDocID(sources[sourceID].iter.getTopKey()); - int docIDCompare = currentDocID.compareTo(docID); - // if this source has advanced beyond the current column qualifier then advance currentCQ and return true - if (docIDCompare < 0) { - currentDocID.set(docID); - advancedCursor = true; - break; - } - // if this source is not yet at the currentCQ then seek in this source - if (docIDCompare > 0) { - // seek forwards - Key seekKey = buildKey(currentPartition, sources[sourceID].term, currentDocID); - sources[sourceID].iter.seek(new Range(seekKey, true, null, false), seekColumnFamilies, inclusive); - continue; - } - // this source is at the current row, in its column family, and at currentCQ - break; - } - } - return advancedCursor; - } - - @Override - public void next() throws IOException { - if (currentPartition == null) { - return; - } - // precondition: the current row is set up and the sources all have the same column qualifier - // while we don't have a match, seek in the source with the smallest column qualifier - sources[0].iter.next(); - advanceToIntersection(); - } - - protected void advanceToIntersection() throws IOException { - boolean cursorChanged = true; - while (cursorChanged) { - // seek all of the sources to at least the highest seen column qualifier in the current row - cursorChanged = false; - for (int i = 0; i < sourcesCount; i++) { - if (currentPartition == null) { - topKey = null; - return; - } - if (seekOneSource(i)) { - cursorChanged = true; - break; - } - } - } - topKey = buildKey(currentPartition, nullText, currentDocID); - } - - public static String stringTopKey(SortedKeyValueIterator iter) { - if (iter.hasTop()) - return iter.getTopKey().toString(); - return ""; - } - - public static final String columnFamiliesOptionName = "columnFamilies"; - public static final String notFlagOptionName = "notFlag"; - - // to be made private - // @see setColumnFamilies - public static String encodeColumns(Text[] columns) { - StringBuilder sb = new StringBuilder(); - for (int i = 0; i < columns.length; i++) { - sb.append(new String(Base64.encodeBase64(TextUtil.getBytes(columns[i])))); - sb.append('\n'); - } - return sb.toString(); - } - - public static String encodeBooleans(boolean[] flags) { - byte[] bytes = new byte[flags.length]; - for (int i = 0; i < flags.length; i++) { - if (flags[i]) - bytes[i] = 1; - else - bytes[i] = 0; - } - return new String(Base64.encodeBase64(bytes)); - } - - private static Text[] decodeColumns(String columns) { - String[] columnStrings = columns.split("\n"); - Text[] columnTexts = new Text[columnStrings.length]; - for (int i = 0; i < columnStrings.length; i++) { - columnTexts[i] = new Text(Base64.decodeBase64(columnStrings[i].getBytes())); - } - return columnTexts; - } - - public static boolean[] decodeBooleans(String flags) { - // return null of there were no flags - if (flags == null) - return null; - - byte[] bytes = Base64.decodeBase64(flags.getBytes()); - boolean[] bFlags = new boolean[bytes.length]; - for (int i = 0; i < bytes.length; i++) { - if (bytes[i] == 1) - bFlags[i] = true; - else - bFlags[i] = false; - } - return bFlags; - } - - @Override - public void init(SortedKeyValueIterator source, Map options, IteratorEnvironment env) throws IOException { - Text[] terms = decodeColumns(options.get(columnFamiliesOptionName)); - boolean[] notFlag = decodeBooleans(options.get(notFlagOptionName)); - - if (terms.length < 2) { - throw new IllegalArgumentException("IntersectionIterator requires two or more columns families"); - } - - // Scan the not flags. - // There must be at least one term that isn't negated - // And we are going to re-order such that the first term is not a ! term - if (notFlag == null) { - notFlag = new boolean[terms.length]; - for (int i = 0; i < terms.length; i++) - notFlag[i] = false; - } - if (notFlag[0]) { - for (int i = 1; i < notFlag.length; i++) { - if (notFlag[i] == false) { - Text swapFamily = new Text(terms[0]); - terms[0].set(terms[i]); - terms[i].set(swapFamily); - notFlag[0] = false; - notFlag[i] = true; - break; - } - } - if (notFlag[0]) { - throw new IllegalArgumentException("IntersectionIterator requires at lest one column family without not"); - } - } - - sources = new TermSource[terms.length]; - sources[0] = new TermSource(source, terms[0]); - for (int i = 1; i < terms.length; i++) { - sources[i] = new TermSource(source.deepCopy(env), terms[i], notFlag[i]); - } - sourcesCount = terms.length; - } - - @Override - public void seek(Range range, Collection seekColumnFamilies, boolean inclusive) throws IOException { - overallRange = new Range(range); - currentPartition = new Text(); - currentDocID.set(emptyByteArray); - - this.seekColumnFamilies = seekColumnFamilies; - this.inclusive = inclusive; - - // seek each of the sources to the right column family within the row given by key - for (int i = 0; i < sourcesCount; i++) { - Key sourceKey; - if (range.getStartKey() != null) { - if (range.getStartKey().getColumnQualifier() != null) { - sourceKey = buildKey(getPartition(range.getStartKey()), sources[i].term, range.getStartKey().getColumnQualifier()); - } else { - sourceKey = buildKey(getPartition(range.getStartKey()), sources[i].term); - } - sources[i].iter.seek(new Range(sourceKey, true, null, false), seekColumnFamilies, inclusive); - } else { - sources[i].iter.seek(range, seekColumnFamilies, inclusive); - } - } - advanceToIntersection(); - } - - public void addSource(SortedKeyValueIterator source, IteratorEnvironment env, Text term, boolean notFlag) { - // Check if we have space for the added Source - if (sources == null) { - sources = new TermSource[1]; - } else { - // allocate space for node, and copy current tree. - // TODO: Should we change this to an ArrayList so that we can just add() ? - TermSource[] localSources = new TermSource[sources.length + 1]; - int currSource = 0; - for (TermSource myTerm : sources) { - // TODO: Do I need to call new here? or can I just re-use the term? - localSources[currSource] = new TermSource(myTerm); - currSource++; - } - sources = localSources; - } - sources[sourcesCount] = new TermSource(source.deepCopy(env), term, notFlag); - sourcesCount++; - } +/** + * This class remains here for backwards compatibility. + * + * @deprecated since 1.4 + * @see org.apache.accumulo.core.iterators.user.IntersectingIterator + */ +public class IntersectingIterator extends org.apache.accumulo.core.iterators.user.IntersectingIterator { - /** - * Encode the columns to be used when iterating. - * - * @param cfg - * @param columns - */ - public static void setColumnFamilies(IteratorSetting cfg, Text[] columns) { - cfg.addOption(IntersectingIterator.columnFamiliesOptionName, IntersectingIterator.encodeColumns(columns)); - } } Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java Thu Nov 17 22:31:36 2011 @@ -27,6 +27,8 @@ import java.util.Map; import java.util.Map.Entry; import java.util.TreeMap; +import org.apache.accumulo.core.client.IteratorSetting; +import org.apache.accumulo.core.client.admin.TableOperations; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.Key; @@ -34,6 +36,7 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.thrift.IterInfo; import org.apache.accumulo.core.iterators.conf.PerColumnIteratorConfig; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.start.classloader.AccumuloClassLoader; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; @@ -62,17 +65,14 @@ public class IteratorUtil { * * @see {@link TableOperations#attachIterators(String, IteratorSetting)} */ - public static Map generateInitialTableProperties(List aggregators) { + public static Map generateAggTableProperties(List aggregators) { - TreeMap props = new TreeMap(); + Map props = new TreeMap(); for (IteratorScope iterScope : IteratorScope.values()) { if (aggregators.size() > 0) { props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".agg", "10," + AggregatingIterator.class.getName()); } - - props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers", "20," + VersioningIterator.class.getName()); - props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers.opt.maxVersions", "1"); } for (PerColumnIteratorConfig ac : aggregators) { @@ -84,6 +84,16 @@ public class IteratorUtil { return props; } + public static Map generateInitialTableProperties() { + TreeMap props = new TreeMap(); + + for (IteratorScope iterScope : IteratorScope.values()) { + props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers", "20," + VersioningIterator.class.getName()); + props.put(Property.TABLE_ITERATOR_PREFIX + iterScope.name() + ".vers.opt.maxVersions", "1"); + } + return props; + } + public static int getMaxPriority(IteratorScope scope, AccumuloConfiguration conf) { List iters = new ArrayList(); parseIterConf(scope, iters, new HashMap>(), conf); Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/LargeRowFilter.java Thu Nov 17 22:31:36 2011 @@ -16,263 +16,12 @@ */ package org.apache.accumulo.core.iterators; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.Map; - -import org.apache.accumulo.core.data.ArrayByteSequence; -import org.apache.accumulo.core.data.ByteSequence; -import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.PartialKey; -import org.apache.accumulo.core.data.Range; -import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.iterators.IteratorEnvironment; -import org.apache.accumulo.core.iterators.OptionDescriber; -import org.apache.accumulo.core.iterators.SortedKeyValueIterator; -import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; -import org.apache.hadoop.io.Text; - /** - * This iterator suppresses rows that exceed a specified number of columns. Once a row exceeds the threshold, a marker is emitted and the row is always - * suppressed by this iterator after that point in time. + * This class remains here for backwards compatibility. * - * This iterator works in a similar way to the RowDeletingIterator. See its javadoc about locality groups. + * @deprecated since 1.4 + * @see org.apache.accumulo.core.iterators.user.LargeRowFilter */ -public class LargeRowFilter implements SortedKeyValueIterator, OptionDescriber { - - public static final Value SUPPRESS_ROW_VALUE = new Value("SUPPRESS_ROW".getBytes()); - - private static final ByteSequence EMPTY = new ArrayByteSequence(new byte[] {}); - - /* key into hash map, value refers to the row supression limit (maxColumns) */ - public static final String MAX_COLUMNS = "max_columns"; - - private SortedKeyValueIterator source; - - // a cache of keys - private ArrayList keys = new ArrayList(); - private ArrayList values = new ArrayList(); - - private int currentPosition; - - private int maxColumns; - - private boolean propogateSuppression = false; - - private Range range; - private Collection columnFamilies; - private boolean inclusive; - private boolean dropEmptyColFams; - - private boolean isSuppressionMarker(Key key, Value val) { - return key.getColumnFamilyData().length() == 0 && key.getColumnQualifierData().length() == 0 && key.getColumnVisibilityData().length() == 0 - && val.equals(SUPPRESS_ROW_VALUE); - } - - private void reseek(Key key) throws IOException { - if (range.afterEndKey(key)) { - range = new Range(range.getEndKey(), true, range.getEndKey(), range.isEndKeyInclusive()); - source.seek(range, columnFamilies, inclusive); - } else { - range = new Range(key, true, range.getEndKey(), range.isEndKeyInclusive()); - source.seek(range, columnFamilies, inclusive); - } - } - - private void consumeRow(ByteSequence row) throws IOException { - // try reading a few and if still not to next row, then seek - int count = 0; - - while (source.hasTop() && source.getTopKey().getRowData().equals(row)) { - source.next(); - count++; - if (count >= 10) { - Key nextRowStart = new Key(new Text(row.toArray())).followingKey(PartialKey.ROW); - reseek(nextRowStart); - count = 0; - } - } - } - - private void addKeyValue(Key k, Value v) { - if (dropEmptyColFams && k.getColumnFamilyData().equals(EMPTY)) { - return; - } - keys.add(new Key(k)); - values.add(new Value(v)); - } - - private void bufferNextRow() throws IOException { - - keys.clear(); - values.clear(); - currentPosition = 0; - - while (source.hasTop() && keys.size() == 0) { - - addKeyValue(source.getTopKey(), source.getTopValue()); - - if (isSuppressionMarker(source.getTopKey(), source.getTopValue())) { - - consumeRow(source.getTopKey().getRowData()); - - } else { - - ByteSequence currentRow = keys.get(0).getRowData(); - source.next(); - - while (source.hasTop() && source.getTopKey().getRowData().equals(currentRow)) { - - addKeyValue(source.getTopKey(), source.getTopValue()); - - if (keys.size() > maxColumns) { - keys.clear(); - values.clear(); - - // when the row is to big, just emit a suppression - // marker - addKeyValue(new Key(new Text(currentRow.toArray())), SUPPRESS_ROW_VALUE); - consumeRow(currentRow); - } else { - source.next(); - } - } - } - - } - } - - private void readNextRow() throws IOException { - - bufferNextRow(); - - while (!propogateSuppression && currentPosition < keys.size() && isSuppressionMarker(keys.get(0), values.get(0))) { - bufferNextRow(); - } - } - - private LargeRowFilter(SortedKeyValueIterator source, boolean propogateSuppression, int maxColumns) { - this.source = source; - this.propogateSuppression = propogateSuppression; - this.maxColumns = maxColumns; - } - - public LargeRowFilter() {} - - @Override - public void init(SortedKeyValueIterator source, Map options, IteratorEnvironment env) throws IOException { - this.source = source; - this.maxColumns = Integer.parseInt(options.get(MAX_COLUMNS)); - this.propogateSuppression = env.getIteratorScope() != IteratorScope.scan; - } - - @Override - public boolean hasTop() { - return currentPosition < keys.size(); - } - - @Override - public void next() throws IOException { - - if (currentPosition >= keys.size()) { - throw new IllegalStateException("Called next() when hasTop() is false"); - } - - currentPosition++; - - if (currentPosition == keys.size()) { - readNextRow(); - } - } - - @Override - public void seek(Range range, Collection columnFamilies, boolean inclusive) throws IOException { - - if (inclusive && !columnFamilies.contains(EMPTY)) { - columnFamilies = new HashSet(columnFamilies); - columnFamilies.add(EMPTY); - dropEmptyColFams = true; - } else if (!inclusive && columnFamilies.contains(EMPTY)) { - columnFamilies = new HashSet(columnFamilies); - columnFamilies.remove(EMPTY); - dropEmptyColFams = true; - } else { - dropEmptyColFams = false; - } - - this.range = range; - this.columnFamilies = columnFamilies; - this.inclusive = inclusive; - - if (range.getStartKey() != null) { - // seek to beginning of row to see if there is a suppression marker - Range newRange = new Range(new Key(range.getStartKey().getRow()), true, range.getEndKey(), range.isEndKeyInclusive()); - source.seek(newRange, columnFamilies, inclusive); - - readNextRow(); - - // it is possible that all or some of the data read for the current - // row is before the start of the range - while (currentPosition < keys.size() && range.beforeStartKey(keys.get(currentPosition))) - currentPosition++; - - if (currentPosition == keys.size()) - readNextRow(); - - } else { - source.seek(range, columnFamilies, inclusive); - readNextRow(); - } - - } - - @Override - public Key getTopKey() { - return keys.get(currentPosition); - } - - @Override - public Value getTopValue() { - return values.get(currentPosition); - } - - @Override - public SortedKeyValueIterator deepCopy(IteratorEnvironment env) { - return new LargeRowFilter(source.deepCopy(env), propogateSuppression, maxColumns); - } - - @Override - public IteratorOptions describeOptions() { - String description = "This iterator suppresses rows that exceed a specified number of columns. Once\n" - + "a row exceeds the threshold, a marker is emitted and the row is always\n" + "suppressed by this iterator after that point in time.\n" - + " This iterator works in a similar way to the RowDeletingIterator. See its\n" + " javadoc about locality groups.\n"; - return new IteratorOptions(this.getClass().getSimpleName(), description, Collections.singletonMap(MAX_COLUMNS, "Number Of Columns To Begin Suppression"), - null); - } - - @Override - public boolean validateOptions(Map options) { - if (options == null || options.size() < 1) { - System.out.println("Bad # of options, must supply: " + MAX_COLUMNS + " as value"); - return false; - } - - if (options.containsKey(MAX_COLUMNS)) { - try { - maxColumns = Integer.parseInt(options.get(MAX_COLUMNS)); - } catch (NumberFormatException e) { - e.printStackTrace(); - return false; - } - } else { - System.out.println("Need to have " + MAX_COLUMNS); - return false; - } - - return true; - } +public class LargeRowFilter extends org.apache.accumulo.core.iterators.user.LargeRowFilter { } Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/VersioningIterator.java Thu Nov 17 22:31:36 2011 @@ -16,141 +16,19 @@ */ package org.apache.accumulo.core.iterators; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; - -import org.apache.accumulo.core.client.IteratorSetting; -import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; -import org.apache.accumulo.core.data.PartialKey; -import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -public class VersioningIterator extends WrappingIterator implements OptionDescriber { - - private Key currentKey = new Key(); - private int numVersions; - private int maxVersions; - - @Override - public VersioningIterator deepCopy(IteratorEnvironment env) { - return new VersioningIterator(this, env); - } - - private VersioningIterator(VersioningIterator other, IteratorEnvironment env) { - setSource(other.getSource().deepCopy(env)); - maxVersions = other.maxVersions; - } - +/** + * This class remains here for backwards compatibility. + * + * @deprecated since 1.4 + * @see org.apache.accumulo.core.iterators.user.VersioningIterator + */ +public class VersioningIterator extends org.apache.accumulo.core.iterators.user.VersioningIterator { public VersioningIterator() {} public VersioningIterator(SortedKeyValueIterator iterator, int maxVersions) { - if (maxVersions < 1) - throw new IllegalArgumentException("maxVersions for versioning iterator must be >= 1"); - this.setSource(iterator); - this.maxVersions = maxVersions; - } - - @Override - public void next() throws IOException { - if (numVersions >= maxVersions) { - skipRowColumn(); - resetVersionCount(); - return; - } - - super.next(); - if (getSource().hasTop()) { - if (getSource().getTopKey().equals(currentKey, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) { - numVersions++; - } else { - resetVersionCount(); - } - } - } - - @Override - public boolean hasTop() { - return super.hasTop(); - } - - @Override - public void seek(Range range, Collection columnFamilies, boolean inclusive) throws IOException { - // do not want to seek to the middle of a row - Range seekRange = IteratorUtil.maximizeStartKeyTimeStamp(range); - - super.seek(seekRange, columnFamilies, inclusive); - resetVersionCount(); - - if (range.getStartKey() != null) { - while (getSource().hasTop() && getSource().getTopKey().compareTo(range.getStartKey(), PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME) < 0) { - // the value has a more recent time stamp, so - // pass it up - // log.debug("skipping "+getTopKey()); - next(); - } - - while (hasTop() && range.beforeStartKey(getTopKey())) { - next(); - } - } - } - - private void resetVersionCount() { - if (super.hasTop()) - currentKey.set(getSource().getTopKey()); - numVersions = 1; - } - - private void skipRowColumn() throws IOException { - Key keyToSkip = currentKey; - super.next(); - - while (getSource().hasTop() && getSource().getTopKey().equals(keyToSkip, PartialKey.ROW_COLFAM_COLQUAL_COLVIS)) { - getSource().next(); - } - } - - @Override - public void init(SortedKeyValueIterator source, Map options, IteratorEnvironment env) throws IOException { - super.init(source, options, env); - this.numVersions = 0; - - String maxVerString = options.get("maxVersions"); - if (maxVerString != null) - this.maxVersions = Integer.parseInt(maxVerString); - else - this.maxVersions = 1; - - if (maxVersions < 1) - throw new IllegalArgumentException("maxVersions for versioning iterator must be >= 1"); - } - - @Override - public IteratorOptions describeOptions() { - return new IteratorOptions("vers", "The VersioningIterator keeps a fixed number of versions for each key", Collections.singletonMap("maxVersions", - "number of versions to keep for a particular key (with differing timestamps)"), null); - } - - private static final String MAXVERSIONS_OPT = "maxVersions"; - - @Override - public boolean validateOptions(Map options) { - int i = Integer.parseInt(options.get(MAXVERSIONS_OPT)); - if (i < 1) - throw new IllegalArgumentException(MAXVERSIONS_OPT + " for versioning iterator must be >= 1"); - return true; - } - - /** - * Encode the maximum number of versions to return onto the ScanIterator - * - * @param cfg - * @param maxVersions - */ - public static void setMaxVersions(IteratorSetting cfg, int maxVersions) { - cfg.addOption(MAXVERSIONS_OPT, Integer.toString(maxVersions)); + super(iterator, maxVersions); } } Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/iterators/conf/PerColumnIteratorConfig.java Thu Nov 17 22:31:36 2011 @@ -16,12 +16,10 @@ */ package org.apache.accumulo.core.iterators.conf; -import org.apache.accumulo.core.iterators.Combiner; import org.apache.hadoop.io.Text; /** * @deprecated since 1.4 - * @see Combiner#addColumn(Text, Text, accumulo.core.client.IteratorSetting) */ public class PerColumnIteratorConfig { Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java Thu Nov 17 22:31:36 2011 @@ -19,7 +19,6 @@ package org.apache.accumulo.core.util.sh import java.io.File; import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Map.Entry; @@ -136,8 +135,7 @@ public class CreateTableCommand extends // context if (cl.hasOption(createTableNoDefaultIters.getOpt())) { - List empty = Collections.emptyList(); - for (String key : IteratorUtil.generateInitialTableProperties(empty).keySet()) + for (String key : IteratorUtil.generateInitialTableProperties().keySet()) shellState.getConnector().tableOperations().removeProperty(tableName, key); } Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GrepCommand.java Thu Nov 17 22:31:36 2011 @@ -24,7 +24,7 @@ import org.apache.accumulo.core.client.A import org.apache.accumulo.core.client.BatchScanner; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.iterators.GrepIterator; +import org.apache.accumulo.core.iterators.user.GrepIterator; import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.util.shell.Shell; import org.apache.commons.cli.CommandLine; Modified: incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java (original) +++ incubator/accumulo/trunk/src/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java Thu Nov 17 22:31:36 2011 @@ -33,11 +33,11 @@ import org.apache.accumulo.core.iterator import org.apache.accumulo.core.iterators.OptionDescriber; import org.apache.accumulo.core.iterators.OptionDescriber.IteratorOptions; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; -import org.apache.accumulo.core.iterators.VersioningIterator; import org.apache.accumulo.core.iterators.aggregation.Aggregator; import org.apache.accumulo.core.iterators.user.AgeOffFilter; import org.apache.accumulo.core.iterators.user.NoVisFilter; import org.apache.accumulo.core.iterators.user.RegExFilter; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.util.shell.Shell; import org.apache.accumulo.core.util.shell.Shell.Command; import org.apache.accumulo.core.util.shell.ShellCommandException; Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java (original) +++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/client/ClientSideIteratorTest.java Thu Nov 17 22:31:36 2011 @@ -35,8 +35,8 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.PartialKey; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.iterators.IntersectingIterator; -import org.apache.accumulo.core.iterators.VersioningIterator; +import org.apache.accumulo.core.iterators.user.IntersectingIterator; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.security.Authorizations; import org.apache.hadoop.io.Text; import org.junit.Test; Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java (original) +++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/FamilyIntersectingIteratorTest.java Thu Nov 17 22:31:36 2011 @@ -35,10 +35,10 @@ import org.apache.accumulo.core.data.Val import org.apache.accumulo.core.file.rfile.RFileTest; import org.apache.accumulo.core.file.rfile.RFileTest.TestRFile; import org.apache.accumulo.core.iterators.FamilyIntersectingIterator; -import org.apache.accumulo.core.iterators.IntersectingIterator; import org.apache.accumulo.core.iterators.IteratorEnvironment; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.iterators.system.MultiIterator; +import org.apache.accumulo.core.iterators.user.IntersectingIterator; import org.apache.hadoop.io.Text; import org.apache.log4j.Level; import org.apache.log4j.Logger; Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java (original) +++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/IntersectingIteratorTest.java Thu Nov 17 22:31:36 2011 @@ -31,11 +31,11 @@ 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.iterators.IntersectingIterator; import org.apache.accumulo.core.iterators.IteratorEnvironment; import org.apache.accumulo.core.iterators.SortedKeyValueIterator; import org.apache.accumulo.core.iterators.SortedMapIterator; import org.apache.accumulo.core.iterators.system.MultiIterator; +import org.apache.accumulo.core.iterators.user.IntersectingIterator; import org.apache.hadoop.io.Text; import org.apache.log4j.Level; import org.apache.log4j.Logger; Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java (original) +++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/LargeRowFilterTest.java Thu Nov 17 22:31:36 2011 @@ -27,10 +27,10 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.PartialKey; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.iterators.LargeRowFilter; import org.apache.accumulo.core.iterators.SortedMapIterator; import org.apache.accumulo.core.iterators.IteratorUtil.IteratorScope; import org.apache.accumulo.core.iterators.system.ColumnFamilySkippingIterator; +import org.apache.accumulo.core.iterators.user.LargeRowFilter; import org.apache.accumulo.core.util.LocalityGroupUtil; import junit.framework.TestCase; Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java (original) +++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/VersioningIteratorTest.java Thu Nov 17 22:31:36 2011 @@ -29,8 +29,8 @@ 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.iterators.SortedMapIterator; -import org.apache.accumulo.core.iterators.VersioningIterator; import org.apache.accumulo.core.iterators.aggregation.LongSummation; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.hadoop.io.Text; public class VersioningIteratorTest extends TestCase { Modified: incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java (original) +++ incubator/accumulo/trunk/src/core/src/test/java/org/apache/accumulo/core/iterators/user/IntersectingIteratorTest2.java Thu Nov 17 22:31:36 2011 @@ -32,7 +32,6 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.Mutation; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.iterators.IntersectingIterator; import org.apache.hadoop.io.Text; import org.junit.Test; Modified: incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java (original) +++ incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/ContinuousQuery.java Thu Nov 17 22:31:36 2011 @@ -31,7 +31,7 @@ import org.apache.accumulo.core.client.Z 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.iterators.IntersectingIterator; +import org.apache.accumulo.core.iterators.user.IntersectingIterator; import org.apache.hadoop.io.Text; /** Modified: incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java (original) +++ incubator/accumulo/trunk/src/examples/src/main/java/org/apache/accumulo/examples/shard/Query.java Thu Nov 17 22:31:36 2011 @@ -27,7 +27,7 @@ import org.apache.accumulo.core.client.Z 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.iterators.IntersectingIterator; +import org.apache.accumulo.core.iterators.user.IntersectingIterator; import org.apache.hadoop.io.Text; /** Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java (original) +++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/master/tableOps/DeleteTable.java Thu Nov 17 22:31:36 2011 @@ -32,7 +32,7 @@ import org.apache.accumulo.core.data.Key import org.apache.accumulo.core.data.KeyExtent; import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.data.Value; -import org.apache.accumulo.core.iterators.GrepIterator; +import org.apache.accumulo.core.iterators.user.GrepIterator; import org.apache.accumulo.core.master.state.tables.TableState; import org.apache.accumulo.core.util.CachedConfiguration; import org.apache.accumulo.server.ServerConstants; Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java (original) +++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Grep.java Thu Nov 17 22:31:36 2011 @@ -29,7 +29,7 @@ import org.apache.accumulo.core.client.I 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.iterators.IntersectingIterator; +import org.apache.accumulo.core.iterators.user.IntersectingIterator; import org.apache.accumulo.core.iterators.user.RegExFilter; import org.apache.accumulo.server.test.randomwalk.State; import org.apache.accumulo.server.test.randomwalk.Test; Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java (original) +++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/test/randomwalk/shard/Search.java Thu Nov 17 22:31:36 2011 @@ -30,7 +30,7 @@ import org.apache.accumulo.core.client.S 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.iterators.IntersectingIterator; +import org.apache.accumulo.core.iterators.user.IntersectingIterator; import org.apache.accumulo.server.test.randomwalk.State; import org.apache.accumulo.server.test.randomwalk.Test; import org.apache.hadoop.io.Text; Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java (original) +++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/upgrade/UpgradeMetadataTable.java Thu Nov 17 22:31:36 2011 @@ -39,9 +39,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.SortedKeyValueIterator; -import org.apache.accumulo.core.iterators.VersioningIterator; import org.apache.accumulo.core.iterators.system.DeletingIterator; import org.apache.accumulo.core.iterators.system.MultiIterator; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.util.CachedConfiguration; import org.apache.accumulo.core.util.ColumnFQ; import org.apache.accumulo.core.util.LocalityGroupUtil; Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java (original) +++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/Initialize.java Thu Nov 17 22:31:36 2011 @@ -35,7 +35,7 @@ import org.apache.accumulo.core.data.Val import org.apache.accumulo.core.file.FileOperations; import org.apache.accumulo.core.file.FileSKVWriter; import org.apache.accumulo.core.file.FileUtil; -import org.apache.accumulo.core.iterators.VersioningIterator; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.master.state.tables.TableState; import org.apache.accumulo.core.master.thrift.MasterGoalState; import org.apache.accumulo.core.util.CachedConfiguration; Modified: incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java URL: http://svn.apache.org/viewvc/incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java?rev=1203397&r1=1203396&r2=1203397&view=diff ============================================================================== --- incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java (original) +++ incubator/accumulo/trunk/src/server/src/main/java/org/apache/accumulo/server/util/OfflineMetadataScanner.java Thu Nov 17 22:31:36 2011 @@ -38,11 +38,11 @@ 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.VersioningIterator; import org.apache.accumulo.core.iterators.system.ColumnQualifierFilter; import org.apache.accumulo.core.iterators.system.DeletingIterator; import org.apache.accumulo.core.iterators.system.MultiIterator; import org.apache.accumulo.core.iterators.system.VisibilityFilter; +import org.apache.accumulo.core.iterators.user.VersioningIterator; import org.apache.accumulo.core.util.CachedConfiguration; import org.apache.accumulo.core.util.LocalityGroupUtil; import org.apache.accumulo.core.util.TextUtil;