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 B73F9D634 for ; Mon, 24 Jun 2013 21:35:02 +0000 (UTC) Received: (qmail 24510 invoked by uid 500); 24 Jun 2013 21:35:02 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 24460 invoked by uid 500); 24 Jun 2013 21:35:02 -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 24453 invoked by uid 99); 24 Jun 2013 21:35:02 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 24 Jun 2013 21:35:02 +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; Mon, 24 Jun 2013 21:34:57 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id 9FA662388A74; Mon, 24 Jun 2013 21:34:38 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1496226 [2/13] - in /accumulo/branches/ACCUMULO-CURATOR: ./ assemble/ conf/examples/1GB/native-standalone/ conf/examples/1GB/standalone/ conf/examples/2GB/native-standalone/ conf/examples/2GB/standalone/ conf/examples/3GB/native-standalone... Date: Mon, 24 Jun 2013 21:34:25 -0000 To: commits@accumulo.apache.org From: vines@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20130624213438.9FA662388A74@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java Mon Jun 24 21:34:20 2013 @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.WeakHashMap; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Instance; @@ -34,6 +33,8 @@ import org.apache.accumulo.core.data.Mut import org.apache.accumulo.core.data.Range; import org.apache.accumulo.core.security.thrift.TCredentials; import org.apache.accumulo.core.util.ArgumentChecker; +import org.apache.accumulo.core.util.MetadataTable; +import org.apache.accumulo.core.util.RootTable; import org.apache.hadoop.io.Text; public abstract class TabletLocator { @@ -90,37 +91,20 @@ public abstract class TabletLocator { private static HashMap locators = new HashMap(); - private static final Text ROOT_TABLET_MDE = KeyExtent.getMetadataEntry(new Text(Constants.METADATA_TABLE_ID), null); - public static synchronized TabletLocator getInstance(Instance instance, Text tableId) { - LocatorKey key = new LocatorKey(instance.getInstanceID(), tableId); + LocatorKey key = new LocatorKey(instance.getInstanceID(), tableId); TabletLocator tl = locators.get(key); - if (tl == null) { MetadataLocationObtainer mlo = new MetadataLocationObtainer(instance); - if (tableId.toString().equals(Constants.METADATA_TABLE_ID)) { - RootTabletLocator rootTabletLocator = new RootTabletLocator(instance); - tl = new TabletLocatorImpl(new Text(Constants.METADATA_TABLE_ID), rootTabletLocator, mlo) { - @Override - public TabletLocation _locateTablet(Text row, boolean skipRow, boolean retry, boolean lock, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, - TableNotFoundException { - // add a special case for the root tablet itself to the cache of information in the root tablet - int comparison_result = row.compareTo(ROOT_TABLET_MDE); - - if ((skipRow && comparison_result < 0) || (!skipRow && comparison_result <= 0)) { - return parent.locateTablet(row, skipRow, retry, credentials); - } - - return super._locateTablet(row, skipRow, retry, lock, credentials); - } - }; + if (tableId.toString().equals(RootTable.ID)) { + tl = new RootTabletLocator(instance); + } else if (tableId.toString().equals(MetadataTable.ID)) { + tl = new TabletLocatorImpl(new Text(MetadataTable.ID), getInstance(instance, new Text(RootTable.ID)), mlo); } else { - TabletLocator rootTabletCache = getInstance(instance, new Text(Constants.METADATA_TABLE_ID)); - tl = new TabletLocatorImpl(tableId, rootTabletCache, mlo); + tl = new TabletLocatorImpl(tableId, getInstance(instance, new Text(MetadataTable.ID)), mlo); } - locators.put(key, tl); } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java Mon Jun 24 21:34:20 2013 @@ -59,6 +59,7 @@ public class TabletLocatorImpl extends T private static class EndRowComparator implements Comparator { + @Override public int compare(Text o1, Text o2) { int ret; @@ -95,10 +96,11 @@ public class TabletLocatorImpl extends T /** * @return null when unable to read information successfully */ - TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException, AccumuloException; - - List lookupTablets(String tserver, Map> map, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException, + TabletLocations lookupTablet(TabletLocation src, Text row, Text stopRow, TabletLocator parent, TCredentials credentials) throws AccumuloSecurityException, AccumuloException; + + List lookupTablets(String tserver, Map> map, TabletLocator parent, TCredentials credentials) + throws AccumuloSecurityException, AccumuloException; } public TabletLocatorImpl(Text table, TabletLocator parent, TabletLocationObtainer tlo) { @@ -111,8 +113,8 @@ public class TabletLocatorImpl extends T } @Override - public void binMutations(List mutations, Map binnedMutations, List failures, TCredentials credentials) throws AccumuloException, - AccumuloSecurityException, TableNotFoundException { + public void binMutations(List mutations, Map binnedMutations, List failures, TCredentials credentials) + throws AccumuloException, AccumuloSecurityException, TableNotFoundException { OpTimer opTimer = null; if (log.isTraceEnabled()) @@ -146,6 +148,7 @@ public class TabletLocatorImpl extends T if (notInCache.size() > 0) { Collections.sort(notInCache, new Comparator() { + @Override public int compare(Mutation o1, Mutation o2) { return WritableComparator.compareBytes(o1.getRow(), 0, o1.getRow().length, o2.getRow(), 0, o2.getRow().length); } @@ -193,8 +196,8 @@ public class TabletLocatorImpl extends T tsm.addMutation(tl.tablet_extent, mutation); } - private List binRanges(List ranges, Map>> binnedRanges, boolean useCache, TCredentials credentials) throws AccumuloException, - AccumuloSecurityException, TableNotFoundException { + private List binRanges(List ranges, Map>> binnedRanges, boolean useCache, TCredentials credentials) + throws AccumuloException, AccumuloSecurityException, TableNotFoundException { List failures = new ArrayList(); List tabletLocations = new ArrayList(); @@ -254,8 +257,8 @@ public class TabletLocatorImpl extends T } @Override - public List binRanges(List ranges, Map>> binnedRanges, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, - TableNotFoundException { + public List binRanges(List ranges, Map>> binnedRanges, TCredentials credentials) throws AccumuloException, + AccumuloSecurityException, TableNotFoundException { /* * For this to be efficient, need to avoid fine grained synchronization and fine grained logging. Therefore methods called by this are not synchronized and @@ -520,8 +523,8 @@ public class TabletLocatorImpl extends T return null; } - protected TabletLocation _locateTablet(Text row, boolean skipRow, boolean retry, boolean lock, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, - TableNotFoundException { + protected TabletLocation _locateTablet(Text row, boolean skipRow, boolean retry, boolean lock, TCredentials credentials) throws AccumuloException, + AccumuloSecurityException, TableNotFoundException { if (skipRow) { row = new Text(row); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java Mon Jun 24 21:34:20 2013 @@ -22,7 +22,6 @@ import java.util.Iterator; import java.util.Map.Entry; import java.util.concurrent.ExecutorService; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.BatchScanner; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.data.Key; @@ -45,7 +44,7 @@ public class TabletServerBatchReader ext private ArrayList ranges; private TCredentials credentials; - private Authorizations authorizations = Constants.NO_AUTHS; + private Authorizations authorizations = Authorizations.EMPTY; private static int nextBatchReaderInstance = 1; Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java Mon Jun 24 21:34:20 2013 @@ -34,9 +34,6 @@ import java.util.concurrent.ExecutorServ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; -import org.apache.accumulo.trace.instrument.TraceRunnable; -import org.apache.accumulo.trace.instrument.Tracer; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Instance; @@ -66,6 +63,8 @@ import org.apache.accumulo.core.util.Byt import org.apache.accumulo.core.util.OpTimer; import org.apache.accumulo.core.util.ThriftUtil; import org.apache.accumulo.core.util.UtilWaitThread; +import org.apache.accumulo.trace.instrument.TraceRunnable; +import org.apache.accumulo.trace.instrument.Tracer; import org.apache.hadoop.io.Text; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -81,7 +80,7 @@ public class TabletServerBatchReaderIter private final Instance instance; private final TCredentials credentials; private final String table; - private Authorizations authorizations = Constants.NO_AUTHS; + private Authorizations authorizations = Authorizations.EMPTY; private final int numThreads; private final ExecutorService queryThreadPool; private final ScannerOptions options; @@ -612,8 +611,8 @@ public class TabletServerBatchReaderIter } static void doLookup(String server, Map> requested, Map> failures, Map> unscanned, - ResultReceiver receiver, List columns, TCredentials credentials, ScannerOptions options, Authorizations authorizations, AccumuloConfiguration conf, - TimeoutTracker timeoutTracker) throws IOException, AccumuloSecurityException, AccumuloServerException { + ResultReceiver receiver, List columns, TCredentials credentials, ScannerOptions options, Authorizations authorizations, + AccumuloConfiguration conf, TimeoutTracker timeoutTracker) throws IOException, AccumuloSecurityException, AccumuloServerException { if (requested.size() == 0) { return; Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java Mon Jun 24 21:34:20 2013 @@ -35,7 +35,6 @@ import java.util.concurrent.ExecutorServ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.BatchWriterConfig; @@ -61,6 +60,7 @@ import org.apache.accumulo.core.tabletse import org.apache.accumulo.core.tabletserver.thrift.NoSuchScanIDException; import org.apache.accumulo.core.tabletserver.thrift.NotServingTabletException; import org.apache.accumulo.core.tabletserver.thrift.TabletClientService; +import org.apache.accumulo.core.util.MetadataTable; import org.apache.accumulo.core.util.SimpleThreadPool; import org.apache.accumulo.core.util.ThriftUtil; import org.apache.accumulo.trace.instrument.Span; @@ -211,6 +211,7 @@ public class TabletServerBatchWriter { if (this.maxLatency != Long.MAX_VALUE) { jtimer.schedule(new TimerTask() { + @Override public void run() { try { synchronized (TabletServerBatchWriter.this) { @@ -460,9 +461,9 @@ public class TabletServerBatchWriter { } } } - + private void updateAuthorizationFailures(Set keySet, SecurityErrorCode code) { - HashMap map = new HashMap(); + HashMap map = new HashMap(); for (KeyExtent ke : keySet) map.put(ke, code); @@ -543,9 +544,6 @@ public class TabletServerBatchWriter { /** * Add mutations that previously failed back into the mix - * - * @param failedMutations - * static final Logger log = Logger.getLogger(TabletServerBatchWriter.class); */ private synchronized void addFailedMutations(MutationSet failedMutations) throws Exception { mutations.addAll(failedMutations); @@ -601,11 +599,11 @@ public class TabletServerBatchWriter { if (rf != null) { if (log.isTraceEnabled()) - log.trace("requeuing " + rf.size() + " failed mutations"); + log.trace("tid=" + Thread.currentThread().getId() + " Requeuing " + rf.size() + " failed mutations"); addFailedMutations(rf); } } catch (Throwable t) { - updateUnknownErrors("Failed to requeue failed mutations " + t.getMessage(), t); + updateUnknownErrors("tid=" + Thread.currentThread().getId() + " Failed to requeue failed mutations " + t.getMessage(), t); cancel(); } } @@ -673,7 +671,7 @@ public class TabletServerBatchWriter { // assume an IOError communicating with !METADATA tablet failedMutations.add(mutationsToProcess); } catch (AccumuloSecurityException e) { - updateAuthorizationFailures(Collections.singletonMap(new KeyExtent(new Text(Constants.METADATA_TABLE_ID), null, null), + updateAuthorizationFailures(Collections.singletonMap(new KeyExtent(new Text(MetadataTable.ID), null, null), SecurityErrorCode.valueOf(e.getSecurityErrorCode().name()))); } catch (TableDeletedException e) { updateUnknownErrors(e.getMessage(), e); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/TimeoutTabletLocator.java Mon Jun 24 21:34:20 2013 @@ -55,9 +55,10 @@ public class TimeoutTabletLocator extend this.locator = locator; this.timeout = timeout; } - + @Override - public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { + public TabletLocation locateTablet(Text row, boolean skipRow, boolean retry, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, + TableNotFoundException { try { TabletLocation ret = locator.locateTablet(row, skipRow, retry, credentials); @@ -75,16 +76,16 @@ public class TimeoutTabletLocator extend } @Override - public void binMutations(List mutations, Map binnedMutations, List failures, TCredentials credentials) throws AccumuloException, - AccumuloSecurityException, TableNotFoundException { + public void binMutations(List mutations, Map binnedMutations, List failures, TCredentials credentials) + throws AccumuloException, AccumuloSecurityException, TableNotFoundException { try { locator.binMutations(mutations, binnedMutations, failures, credentials); - + if (failures.size() == mutations.size()) failed(); else succeeded(); - + } catch (AccumuloException ae) { failed(); throw ae; @@ -94,10 +95,10 @@ public class TimeoutTabletLocator extend /** * */ - + @Override - public List binRanges(List ranges, Map>> binnedRanges, TCredentials credentials) throws AccumuloException, AccumuloSecurityException, - TableNotFoundException { + public List binRanges(List ranges, Map>> binnedRanges, TCredentials credentials) throws AccumuloException, + AccumuloSecurityException, TableNotFoundException { try { List ret = locator.binRanges(ranges, binnedRanges, credentials); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/ClientService.java Mon Jun 24 21:34:20 2013 @@ -5082,7 +5082,7 @@ import org.slf4j.LoggerFactory; struct.files = new ArrayList(_list8.size); for (int _i9 = 0; _i9 < _list8.size; ++_i9) { - String _elem10; // required + String _elem10; // optional _elem10 = iprot.readString(); struct.files.add(_elem10); } @@ -5259,7 +5259,7 @@ import org.slf4j.LoggerFactory; struct.files = new ArrayList(_list13.size); for (int _i14 = 0; _i14 < _list13.size; ++_i14) { - String _elem15; // required + String _elem15; // optional _elem15 = iprot.readString(); struct.files.add(_elem15); } @@ -5746,7 +5746,7 @@ import org.slf4j.LoggerFactory; struct.success = new ArrayList(_list16.size); for (int _i17 = 0; _i17 < _list16.size; ++_i17) { - String _elem18; // required + String _elem18; // optional _elem18 = iprot.readString(); struct.success.add(_elem18); } @@ -5867,7 +5867,7 @@ import org.slf4j.LoggerFactory; struct.success = new ArrayList(_list21.size); for (int _i22 = 0; _i22 < _list21.size; ++_i22) { - String _elem23; // required + String _elem23; // optional _elem23 = iprot.readString(); struct.success.add(_elem23); } @@ -7808,7 +7808,7 @@ import org.slf4j.LoggerFactory; struct.tables = new HashSet(2*_set24.size); for (int _i25 = 0; _i25 < _set24.size; ++_i25) { - String _elem26; // required + String _elem26; // optional _elem26 = iprot.readString(); struct.tables.add(_elem26); } @@ -7909,7 +7909,7 @@ import org.slf4j.LoggerFactory; struct.tables = new HashSet(2*_set29.size); for (int _i30 = 0; _i30 < _set29.size; ++_i30) { - String _elem31; // required + String _elem31; // optional _elem31 = iprot.readString(); struct.tables.add(_elem31); } @@ -8393,7 +8393,7 @@ import org.slf4j.LoggerFactory; struct.success = new ArrayList(_list32.size); for (int _i33 = 0; _i33 < _list32.size; ++_i33) { - TDiskUsage _elem34; // required + TDiskUsage _elem34; // optional _elem34 = new TDiskUsage(); _elem34.read(iprot); struct.success.add(_elem34); @@ -8515,7 +8515,7 @@ import org.slf4j.LoggerFactory; struct.success = new ArrayList(_list37.size); for (int _i38 = 0; _i38 < _list37.size; ++_i38) { - TDiskUsage _elem39; // required + TDiskUsage _elem39; // optional _elem39 = new TDiskUsage(); _elem39.read(iprot); struct.success.add(_elem39); @@ -9392,7 +9392,7 @@ import org.slf4j.LoggerFactory; struct.success = new HashSet(2*_set40.size); for (int _i41 = 0; _i41 < _set40.size; ++_i41) { - String _elem42; // required + String _elem42; // optional _elem42 = iprot.readString(); struct.success.add(_elem42); } @@ -9493,7 +9493,7 @@ import org.slf4j.LoggerFactory; struct.success = new HashSet(2*_set45.size); for (int _i46 = 0; _i46 < _set45.size; ++_i46) { - String _elem47; // required + String _elem47; // optional _elem47 = iprot.readString(); struct.success.add(_elem47); } @@ -15015,7 +15015,7 @@ import org.slf4j.LoggerFactory; struct.authorizations = new ArrayList(_list48.size); for (int _i49 = 0; _i49 < _list48.size; ++_i49) { - ByteBuffer _elem50; // required + ByteBuffer _elem50; // optional _elem50 = iprot.readBinary(); struct.authorizations.add(_elem50); } @@ -15143,7 +15143,7 @@ import org.slf4j.LoggerFactory; struct.authorizations = new ArrayList(_list53.size); for (int _i54 = 0; _i54 < _list53.size; ++_i54) { - ByteBuffer _elem55; // required + ByteBuffer _elem55; // optional _elem55 = iprot.readBinary(); struct.authorizations.add(_elem55); } @@ -16467,7 +16467,7 @@ import org.slf4j.LoggerFactory; struct.success = new ArrayList(_list56.size); for (int _i57 = 0; _i57 < _list56.size; ++_i57) { - ByteBuffer _elem58; // required + ByteBuffer _elem58; // optional _elem58 = iprot.readBinary(); struct.success.add(_elem58); } @@ -16568,7 +16568,7 @@ import org.slf4j.LoggerFactory; struct.success = new ArrayList(_list61.size); for (int _i62 = 0; _i62 < _list61.size; ++_i62) { - ByteBuffer _elem63; // required + ByteBuffer _elem63; // optional _elem63 = iprot.readBinary(); struct.success.add(_elem63); } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TDiskUsage.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TDiskUsage.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TDiskUsage.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/impl/thrift/TDiskUsage.java Mon Jun 24 21:34:20 2013 @@ -436,7 +436,7 @@ import org.slf4j.LoggerFactory; struct.tables = new ArrayList(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { - String _elem2; // required + String _elem2; // optional _elem2 = iprot.readString(); struct.tables.add(_elem2); } @@ -534,7 +534,7 @@ import org.slf4j.LoggerFactory; struct.tables = new ArrayList(_list5.size); for (int _i6 = 0; _i6 < _list5.size; ++_i6) { - String _elem7; // required + String _elem7; // optional _elem7 = iprot.readString(); struct.tables.add(_elem7); } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java Mon Jun 24 21:34:20 2013 @@ -28,7 +28,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.ClientSideIteratorScanner; @@ -57,6 +56,7 @@ import org.apache.accumulo.core.master.s import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.CredentialHelper; import org.apache.accumulo.core.security.thrift.TCredentials; +import org.apache.accumulo.core.util.MetadataTable; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.hadoop.io.Text; @@ -686,11 +686,11 @@ public abstract class InputFormatBase entry = row.next(); Key key = entry.getKey(); - if (key.getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) { + if (key.getColumnFamily().equals(MetadataTable.LAST_LOCATION_COLUMN_FAMILY)) { last = entry.getValue().toString(); } - if (key.getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY) - || key.getColumnFamily().equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) { + if (key.getColumnFamily().equals(MetadataTable.CURRENT_LOCATION_COLUMN_FAMILY) + || key.getColumnFamily().equals(MetadataTable.FUTURE_LOCATION_COLUMN_FAMILY)) { location = entry.getValue().toString(); } - if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) { + if (MetadataTable.PREV_ROW_COLUMN.hasColumns(key)) { extent = new KeyExtent(key.getRow(), entry.getValue()); } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java Mon Jun 24 21:34:20 2013 @@ -36,7 +36,6 @@ import java.util.Map.Entry; import java.util.Set; import java.util.StringTokenizer; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.ClientSideIteratorScanner; @@ -68,6 +67,7 @@ import org.apache.accumulo.core.master.s import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.CredentialHelper; import org.apache.accumulo.core.security.thrift.TCredentials; +import org.apache.accumulo.core.util.MetadataTable; import org.apache.accumulo.core.util.Pair; import org.apache.accumulo.core.util.UtilWaitThread; import org.apache.hadoop.conf.Configuration; @@ -710,11 +710,11 @@ public abstract class InputFormatBase entry = row.next(); Key key = entry.getKey(); - if (key.getColumnFamily().equals(Constants.METADATA_LAST_LOCATION_COLUMN_FAMILY)) { + if (key.getColumnFamily().equals(MetadataTable.LAST_LOCATION_COLUMN_FAMILY)) { last = entry.getValue().toString(); } - if (key.getColumnFamily().equals(Constants.METADATA_CURRENT_LOCATION_COLUMN_FAMILY) - || key.getColumnFamily().equals(Constants.METADATA_FUTURE_LOCATION_COLUMN_FAMILY)) { + if (key.getColumnFamily().equals(MetadataTable.CURRENT_LOCATION_COLUMN_FAMILY) + || key.getColumnFamily().equals(MetadataTable.FUTURE_LOCATION_COLUMN_FAMILY)) { location = entry.getValue().toString(); } - if (Constants.METADATA_PREV_ROW_COLUMN.hasColumns(key)) { + if (MetadataTable.PREV_ROW_COLUMN.hasColumns(key)) { extent = new KeyExtent(key.getRow(), entry.getValue()); } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java Mon Jun 24 21:34:20 2013 @@ -29,7 +29,6 @@ import java.util.List; import java.util.Set; import java.util.StringTokenizer; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.AccumuloException; import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.ClientSideIteratorScanner; @@ -138,7 +137,7 @@ public class InputConfigurator extends C */ public static Authorizations getScanAuthorizations(Class implementingClass, Configuration conf) { String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS)); - return authString == null ? Constants.NO_AUTHS : new Authorizations(authString.getBytes()); + return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes()); } /** Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java Mon Jun 24 21:34:20 2013 @@ -22,7 +22,6 @@ import java.util.HashMap; import java.util.Map; import java.util.SortedSet; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.BatchScanner; import org.apache.accumulo.core.client.admin.TimeType; import org.apache.accumulo.core.client.security.tokens.PasswordToken; @@ -30,6 +29,8 @@ import org.apache.accumulo.core.data.Mut import org.apache.accumulo.core.security.Authorizations; import org.apache.accumulo.core.security.SystemPermission; import org.apache.accumulo.core.security.TablePermission; +import org.apache.accumulo.core.util.MetadataTable; +import org.apache.accumulo.core.util.RootTable; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.Text; @@ -44,10 +45,11 @@ public class MockAccumulo { } { - MockUser root = new MockUser("root", new PasswordToken(new byte[0]), Constants.NO_AUTHS); + MockUser root = new MockUser("root", new PasswordToken(new byte[0]), Authorizations.EMPTY); root.permissions.add(SystemPermission.SYSTEM); users.put(root.name, root); - createTable("root", Constants.METADATA_TABLE_NAME, true, TimeType.LOGICAL); + createTable("root", RootTable.NAME, true, TimeType.LOGICAL); + createTable("root", MetadataTable.NAME, true, TimeType.LOGICAL); } public FileSystem getFileSystem() { @@ -88,8 +90,8 @@ public class MockAccumulo { public Collection getSplits(String tableName) { return tables.get(tableName).getSplits(); } - + public void merge(String tableName, Text start, Text end) { tables.get(tableName).merge(start, end); - } + } } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchWriter.java Mon Jun 24 21:34:20 2013 @@ -19,6 +19,7 @@ package org.apache.accumulo.core.client. import org.apache.accumulo.core.client.BatchWriter; import org.apache.accumulo.core.client.MutationsRejectedException; import org.apache.accumulo.core.data.Mutation; +import org.apache.accumulo.core.util.ArgumentChecker; public class MockBatchWriter implements BatchWriter { @@ -32,11 +33,13 @@ public class MockBatchWriter implements @Override public void addMutation(Mutation m) throws MutationsRejectedException { + ArgumentChecker.notNull(m); acu.addMutation(tablename, m); } @Override public void addMutations(Iterable iterable) throws MutationsRejectedException { + ArgumentChecker.notNull(iterable); for (Mutation m : iterable) { acu.addMutation(tablename, m); } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java Mon Jun 24 21:34:20 2013 @@ -33,78 +33,40 @@ import org.apache.accumulo.start.classlo public class MockInstanceOperations implements InstanceOperations { MockAccumulo acu; - /** - * @param acu - */ public MockInstanceOperations(MockAccumulo acu) { this.acu = acu; } - /* - * (non-Javadoc) - * - * @see org.apache.accumulo.core.client.admin.InstanceOperations#setProperty(java.lang.String, java.lang.String) - */ @Override public void setProperty(String property, String value) throws AccumuloException, AccumuloSecurityException { acu.setProperty(property, value); } - /* - * (non-Javadoc) - * - * @see org.apache.accumulo.core.client.admin.InstanceOperations#removeProperty(java.lang.String) - */ @Override public void removeProperty(String property) throws AccumuloException, AccumuloSecurityException { acu.removeProperty(property); } - /* - * (non-Javadoc) - * - * @see org.apache.accumulo.core.client.admin.InstanceOperations#getSystemConfiguration() - */ @Override public Map getSystemConfiguration() throws AccumuloException, AccumuloSecurityException { return acu.systemProperties; } - /* - * (non-Javadoc) - * - * @see org.apache.accumulo.core.client.admin.InstanceOperations#getSiteConfiguration() - */ @Override public Map getSiteConfiguration() throws AccumuloException, AccumuloSecurityException { return acu.systemProperties; } - /* - * (non-Javadoc) - * - * @see org.apache.accumulo.core.client.admin.InstanceOperations#getTabletServers() - */ @Override public List getTabletServers() { return new ArrayList(); } - /* - * (non-Javadoc) - * - * @see org.apache.accumulo.core.client.admin.InstanceOperations#getActiveScans(java.lang.String) - */ @Override public List getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException { return new ArrayList(); } - /* - * (non-Javadoc) - * - * @see org.apache.accumulo.core.client.admin.InstanceOperations#testClassLoad(java.lang.String, java.lang.String) - */ @Override public boolean testClassLoad(String className, String asTypeName) throws AccumuloException, AccumuloSecurityException { try { @@ -116,11 +78,6 @@ public class MockInstanceOperations impl return true; } - /* - * (non-Javadoc) - * - * @see org.apache.accumulo.core.client.admin.InstanceOperations#getActiveCompactions(java.lang.String) - */ @Override public List getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException { return new ArrayList(); @@ -128,6 +85,6 @@ public class MockInstanceOperations impl @Override public void ping(String tserver) throws AccumuloException { - + } } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockTable.java Mon Jun 24 21:34:20 2013 @@ -101,6 +101,8 @@ public class MockTable { } synchronized void addMutation(Mutation m) { + if (m.size() == 0) + throw new IllegalArgumentException("Can not add empty mutations"); long now = System.currentTimeMillis(); mutationCount++; for (ColumnUpdate u : m.getUpdates()) { Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/client/mock/MockTableOperations.java Mon Jun 24 21:34:20 2013 @@ -306,11 +306,7 @@ public class MockTableOperations extends public List getDiskUsage(Set tables) throws AccumuloException, AccumuloSecurityException { List diskUsages = new ArrayList(); - for(String table : tables) { - TreeSet tree = new TreeSet(); - tree.add(table); - diskUsages.add(new DiskUsage(tree, 1l)); - } + diskUsages.add(new DiskUsage(new TreeSet(tables), 0l)); return diskUsages; } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/conf/Property.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/conf/Property.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/conf/Property.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/conf/Property.java Mon Jun 24 21:34:20 2013 @@ -34,27 +34,35 @@ import org.apache.log4j.Logger; public enum Property { // Crypto-related properties - CRYPTO_PREFIX("crypto.", null, PropertyType.PREFIX, "Properties in this category related to the configuration of both default and custom crypto modules.", - true, false), + @Experimental + CRYPTO_PREFIX("crypto.", null, PropertyType.PREFIX, "Properties in this category related to the configuration of both default and custom crypto modules."), + @Experimental CRYPTO_MODULE_CLASS("crypto.module.class", "NullCryptoModule", PropertyType.STRING, "Fully qualified class name of the class that implements the CryptoModule interface, to be used in setting up encryption at rest for the WAL and " - + "(future) other parts of the code.", true, false), - CRYPTO_CIPHER_SUITE("crypto.cipher.suite", "NullCipher", PropertyType.STRING, "Describes the cipher suite to use for the write-ahead log", true, false), + + "(future) other parts of the code."), + @Experimental + CRYPTO_CIPHER_SUITE("crypto.cipher.suite", "NullCipher", PropertyType.STRING, "Describes the cipher suite to use for the write-ahead log"), + @Experimental CRYPTO_CIPHER_ALGORITHM_NAME("crypto.cipher.algorithm.name", "NullCipher", PropertyType.STRING, - "States the name of the algorithm used in the corresponding cipher suite. Do not make these different, unless you enjoy mysterious exceptions and bugs.", - true, false), + "States the name of the algorithm used in the corresponding cipher suite. Do not make these different, unless you enjoy mysterious exceptions and bugs."), + @Experimental CRYPTO_CIPHER_KEY_LENGTH("crypto.cipher.key.length", "128", PropertyType.STRING, - "Specifies the key length *in bits* to use for the symmetric key, should probably be 128 or 256 unless you really know what you're doing", true, false), + "Specifies the key length *in bits* to use for the symmetric key, should probably be 128 or 256 unless you really know what you're doing"), + @Experimental CRYPTO_SECURE_RNG("crypto.secure.rng", "SHA1PRNG", PropertyType.STRING, - "States the secure random number generator to use, and defaults to the built-in Sun SHA1PRNG", true, false), + "States the secure random number generator to use, and defaults to the built-in Sun SHA1PRNG"), + @Experimental CRYPTO_SECURE_RNG_PROVIDER("crypto.secure.rng.provider", "SUN", PropertyType.STRING, - "States the secure random number generator provider to use, and defaults to the built-in SUN provider", true, false), + "States the secure random number generator provider to use, and defaults to the built-in SUN provider"), + @Experimental CRYPTO_SECRET_KEY_ENCRYPTION_STRATEGY_CLASS("crypto.secret.key.encryption.strategy.class", "NullSecretKeyEncryptionStrategy", PropertyType.STRING, - "The class Accumulo should use for its key encryption strategy.", true, false), + "The class Accumulo should use for its key encryption strategy."), + @Experimental CRYPTO_DEFAULT_KEY_STRATEGY_HDFS_URI("crypto.default.key.strategy.hdfs.uri", "", PropertyType.STRING, - "The URL Accumulo should use to connect to DFS. If this is blank, Accumulo will obtain this information from the Hadoop configuration", true, false), + "The URL Accumulo should use to connect to DFS. If this is blank, Accumulo will obtain this information from the Hadoop configuration"), + @Experimental CRYPTO_DEFAULT_KEY_STRATEGY_KEY_LOCATION("crypto.default.key.strategy.key.location", "/accumulo/crypto/secret/keyEncryptionKey", PropertyType.ABSOLUTEPATH, - "The absolute path of where to store the key encryption key within HDFS.", true, false), + "The absolute path of where to store the key encryption key within HDFS."), // instance properties (must be the same for every node in an instance) INSTANCE_PREFIX("instance.", null, PropertyType.PREFIX, @@ -66,10 +74,12 @@ public enum Property { "The url accumulo should use to connect to DFS. If this is empty, accumulo will obtain this information from the hadoop configuration."), INSTANCE_DFS_DIR("instance.dfs.dir", "/accumulo", PropertyType.ABSOLUTEPATH, "HDFS directory in which accumulo instance will run. Do not change after accumulo is initialized."), + @Sensitive INSTANCE_SECRET("instance.secret", "DEFAULT", PropertyType.STRING, "A secret unique to a given instance that all servers must know in order to communicate with one another." + " Change it before initialization. To change it later use ./bin/accumulo accumulo.server.util.ChangeSecret [oldpasswd] [newpasswd], " + " and then update conf/accumulo-site.xml everywhere."), + INSTANCE_VOLUMES("instance.volumes", "", PropertyType.STRING, "A list of volumes to use. By default, this will be the namenode in the hadoop configuration in the accumulo classpath."), INSTANCE_SECURITY_AUTHENTICATOR("instance.security.authenticator", "org.apache.accumulo.server.security.handler.ZKAuthenticator", PropertyType.CLASSNAME, "The authenticator class that accumulo will use to determine if a user has privilege to perform an action"), INSTANCE_SECURITY_AUTHORIZOR("instance.security.authorizor", "org.apache.accumulo.server.security.handler.ZKAuthorizor", PropertyType.CLASSNAME, @@ -91,6 +101,8 @@ public enum Property { GENERAL_KERBEROS_PRINCIPAL("general.kerberos.principal", "", PropertyType.STRING, "Name of the kerberos principal to use. _HOST will automatically be " + "replaced by the machines hostname in the hostname portion of the principal. Leave blank if not using kerberoized hdfs"), GENERAL_MAX_MESSAGE_SIZE("tserver.server.message.size.max", "1G", PropertyType.MEMORY, "The maximum size of a message that can be sent to a tablet server."), + GENERAL_VOLUME_CHOOSER("general.volume.chooser", "org.apache.accumulo.server.fs.RandomVolumeChooser", PropertyType.CLASSNAME, "The class that will be used to select which volume will be used to create new files."), + // properties that are specific to master server behavior MASTER_PREFIX("master.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the master server"), @@ -207,7 +219,7 @@ public enum Property { "The property only needs to be set if upgrading from 1.4 which used to store write-ahead logs on the local filesystem. In 1.5 write-ahead logs are " + "stored in DFS. When 1.5 is started for the first time it will copy any 1.4 write ahead logs into DFS. It is possible to specify a " + "comma-separated list of directories."), - + // accumulo garbage collector properties GC_PREFIX("gc.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of the accumulo garbage collector."), GC_CYCLE_START("gc.cycle.start", "30s", PropertyType.TIMEDURATION, "Time to wait before attempting to garbage collect any old files."), @@ -225,16 +237,24 @@ public enum Property { MONITOR_BANNER_COLOR("monitor.banner.color", "#c4c4c4", PropertyType.STRING, "The color of the banner text displayed on the monitor page."), MONITOR_BANNER_BACKGROUND("monitor.banner.background", "#304065", PropertyType.STRING, "The background color of the banner text displayed on the monitor page."), - MONITOR_SSL_KEYSTORE("monitor.ssl.keyStore", "", PropertyType.PATH, "The keystore for enabling monitor SSL.", true, false), - MONITOR_SSL_KEYSTOREPASS("monitor.ssl.keyStorePassword", "", PropertyType.STRING, "The keystore password for enabling monitor SSL.", true, false), - MONITOR_SSL_TRUSTSTORE("monitor.ssl.trustStore", "", PropertyType.PATH, "The truststore for enabling monitor SSL.", true, false), - MONITOR_SSL_TRUSTSTOREPASS("monitor.ssl.trustStorePassword", "", PropertyType.STRING, "The truststore password for enabling monitor SSL.", true, false), + @Experimental + MONITOR_SSL_KEYSTORE("monitor.ssl.keyStore", "", PropertyType.PATH, "The keystore for enabling monitor SSL."), + @Experimental + @Sensitive + MONITOR_SSL_KEYSTOREPASS("monitor.ssl.keyStorePassword", "", PropertyType.STRING, "The keystore password for enabling monitor SSL."), + @Experimental + MONITOR_SSL_TRUSTSTORE("monitor.ssl.trustStore", "", PropertyType.PATH, "The truststore for enabling monitor SSL."), + @Experimental + @Sensitive + MONITOR_SSL_TRUSTSTOREPASS("monitor.ssl.trustStorePassword", "", PropertyType.STRING, "The truststore password for enabling monitor SSL."), TRACE_PREFIX("trace.", null, PropertyType.PREFIX, "Properties in this category affect the behavior of distributed tracing."), TRACE_PORT("trace.port.client", "12234", PropertyType.PORT, "The listening port for the trace server"), TRACE_TABLE("trace.table", "trace", PropertyType.STRING, "The name of the table to store distributed traces"), TRACE_USER("trace.user", "root", PropertyType.STRING, "The name of the user to store distributed traces"), + @Sensitive TRACE_PASSWORD("trace.password", "secret", PropertyType.STRING, "The password for the user used to store distributed traces"), + @Sensitive TRACE_TOKEN_PROPERTY_PREFIX("trace.token.property", null, PropertyType.PREFIX, "The prefix used to create a token for storing distributed traces. For each propetry required by trace.token.type, place this prefix in front of it."), TRACE_TOKEN_TYPE("trace.token.type", PasswordToken.class.getName(), PropertyType.CLASSNAME, "An AuthenticationToken type supported by the authorizer"), @@ -344,27 +364,27 @@ public enum Property { + "You can enable post delegation for a context, which will load classes from the context first instead of the parent first. " + "Do this by setting general.vfs.context.classpath.<name>.delegation=post, where <name> is your context name. " + "If delegation is not specified, it defaults to loading from parent classloader first."), + @Interpolated VFS_CLASSLOADER_CACHE_DIR(AccumuloVFSClassLoader.VFS_CACHE_DIR, "${java.io.tmpdir}" + File.separator + "accumulo-vfs-cache-${user.name}", PropertyType.ABSOLUTEPATH, "Directory to use for the vfs cache. The cache will keep a soft reference to all of the classes loaded in the VM." - + " This should be on local disk on each node with sufficient space. It defaults to ${java.io.tmpdir}/accumulo-vfs-cache-${user.name}", false, true); + + " This should be on local disk on each node with sufficient space. It defaults to ${java.io.tmpdir}/accumulo-vfs-cache-${user.name}"), + + @Interpolated + @Experimental + GENERAL_MAVEN_PROJECT_BASEDIR(AccumuloClassLoader.MAVEN_PROJECT_BASEDIR_PROPERTY_NAME, AccumuloClassLoader.DEFAULT_MAVEN_PROJECT_BASEDIR_VALUE, + PropertyType.ABSOLUTEPATH, "Set this to automatically add maven target/classes directories to your dynamic classpath"), + + ; private String key, defaultValue, description; private PropertyType type; - private boolean experimental; - private boolean interpolated; static Logger log = Logger.getLogger(Property.class); - private Property(String name, String defaultValue, PropertyType type, String description, boolean experimental, boolean interpolated) { + private Property(String name, String defaultValue, PropertyType type, String description) { this.key = name; this.defaultValue = defaultValue; this.description = description; this.type = type; - this.experimental = experimental; - this.interpolated = interpolated; - } - - private Property(String name, String defaultValue, PropertyType type, String description) { - this(name, defaultValue, type, description, false, false); } @Override @@ -381,7 +401,7 @@ public enum Property { } public String getDefaultValue() { - if (this.interpolated) { + if (isInterpolated()) { PropertiesConfiguration pconf = new PropertiesConfiguration(); pconf.append(new SystemConfiguration()); pconf.addProperty("hack_default_value", this.defaultValue); @@ -403,8 +423,54 @@ public enum Property { return this.description; } + private boolean isInterpolated() { + return hasAnnotation(Interpolated.class) || hasPrefixWithAnnotation(getKey(), Interpolated.class); + } + public boolean isExperimental() { - return experimental; + return hasAnnotation(Experimental.class) || hasPrefixWithAnnotation(getKey(), Experimental.class); + } + + public boolean isDeprecated() { + return hasAnnotation(Deprecated.class) || hasPrefixWithAnnotation(getKey(), Deprecated.class); + } + + public boolean isSensitive() { + return hasAnnotation(Sensitive.class) || hasPrefixWithAnnotation(getKey(), Sensitive.class); + } + + public static boolean isSensitive(String key) { + return hasPrefixWithAnnotation(key, Sensitive.class); + } + + private boolean hasAnnotation(Class annotationType) { + Logger log = Logger.getLogger(getClass()); + try { + for (Annotation a : getClass().getField(name()).getAnnotations()) + if (annotationType.isInstance(a)) + return true; + } catch (SecurityException e) { + log.error(e, e); + } catch (NoSuchFieldException e) { + log.error(e, e); + } + return false; + } + + private static boolean hasPrefixWithAnnotation(String key, Class annotationType) { + // relies on side-effects of isValidPropertyKey to populate validPrefixes + if (isValidPropertyKey(key)) { + // check if property exists on its own and has the annotation + if (Property.getPropertyByKey(key) != null) + return getPropertyByKey(key).hasAnnotation(annotationType); + // can't find the property, so check the prefixes + boolean prefixHasAnnotation = false; + for (String prefix : validPrefixes) + if (key.startsWith(prefix)) + prefixHasAnnotation = prefixHasAnnotation || getPropertyByKey(prefix).hasAnnotation(annotationType); + return prefixHasAnnotation; + } + return false; } private static HashSet validTableProperties = null; @@ -485,20 +551,6 @@ public enum Property { || key.equals(Property.TABLE_LOAD_BALANCER.getKey()); } - public boolean isDeprecated() { - Logger log = Logger.getLogger(getClass()); - try { - for (Annotation a : getClass().getField(name()).getAnnotations()) - if (a instanceof Deprecated) - return true; - } catch (SecurityException e) { - log.error(e, e); - } catch (NoSuchFieldException e) { - log.error(e, e); - } - return false; - } - public static T createInstanceFromPropertyName(AccumuloConfiguration conf, Property property, Class base, T defaultInstance) { String clazzName = conf.get(property); T instance = null; Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/KeyExtent.java Mon Jun 24 21:34:20 2013 @@ -39,9 +39,10 @@ import java.util.TreeSet; import java.util.UUID; import java.util.WeakHashMap; -import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.data.thrift.TKeyExtent; import org.apache.accumulo.core.util.ByteBufferUtil; +import org.apache.accumulo.core.util.MetadataTable; +import org.apache.accumulo.core.util.RootTable; import org.apache.accumulo.core.util.TextUtil; import org.apache.hadoop.io.BinaryComparable; import org.apache.hadoop.io.Text; @@ -250,6 +251,7 @@ public class KeyExtent implements Writab * Populates the extents data fields from a DataInput object * */ + @Override public void readFields(DataInput in) throws IOException { Text tid = new Text(); tid.readFields(in); @@ -279,6 +281,7 @@ public class KeyExtent implements Writab * Writes this extent's data fields to a DataOutput object * */ + @Override public void write(DataOutput out) throws IOException { getTableId().write(out); if (getEndRow() != null) { @@ -395,7 +398,7 @@ public class KeyExtent implements Writab public static Mutation getPrevRowUpdateMutation(KeyExtent ke) { Mutation m = new Mutation(ke.getMetadataEntry()); - Constants.METADATA_PREV_ROW_COLUMN.put(m, encodePrevEndRow(ke.getPrevEndRow())); + MetadataTable.PREV_ROW_COLUMN.put(m, encodePrevEndRow(ke.getPrevEndRow())); return m; } @@ -403,6 +406,7 @@ public class KeyExtent implements Writab * Compares extents based on rows * */ + @Override public int compareTo(KeyExtent other) { int result = getTableId().compareTo(other.getTableId()); @@ -754,9 +758,6 @@ public class KeyExtent implements Writab : TextUtil.getByteBuffer(textPrevEndRow)); } - /** - * @param prevExtent - */ public boolean isPreviousExtent(KeyExtent prevExtent) { if (prevExtent == null) return getPrevEndRow() == null; @@ -774,10 +775,10 @@ public class KeyExtent implements Writab } public boolean isMeta() { - return getTableId().toString().equals(Constants.METADATA_TABLE_ID); + return getTableId().toString().equals(MetadataTable.ID) || isRootTablet(); } public boolean isRootTablet() { - return this.compareTo(Constants.ROOT_TABLET_EXTENT) == 0; + return getTableId().toString().equals(RootTable.ID); } } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/Value.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/Value.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/Value.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/Value.java Mon Jun 24 21:34:20 2013 @@ -22,7 +22,6 @@ import static org.apache.accumulo.core.u import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.io.UnsupportedEncodingException; import java.nio.ByteBuffer; import java.util.List; @@ -30,7 +29,6 @@ import org.apache.accumulo.core.Constant import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.io.WritableComparator; -import org.apache.log4j.Logger; /** * A byte sequence that is usable as a key or value. Based on {@link org.apache.hadoop.io.BytesWritable} only this class is NOT resizable and DOES NOT @@ -39,7 +37,6 @@ import org.apache.log4j.Logger; */ public class Value implements WritableComparable { protected byte[] value; - private static final Logger log = Logger.getLogger(Value.class); /** * Create a zero-size sequence. @@ -142,12 +139,13 @@ public class Value implements WritableCo return this.value.length; } + @Override public void readFields(final DataInput in) throws IOException { this.value = new byte[in.readInt()]; in.readFully(this.value, 0, this.value.length); } - /** {@inheritDoc} */ + @Override public void write(final DataOutput out) throws IOException { out.writeInt(this.value.length); out.write(this.value, 0, this.value.length); @@ -155,7 +153,6 @@ public class Value implements WritableCo // Below methods copied from BytesWritable - /** {@inheritDoc} */ @Override public int hashCode() { return WritableComparator.hashBytes(value, this.value.length); @@ -168,6 +165,7 @@ public class Value implements WritableCo * The other bytes writable * @return Positive if left is bigger than right, 0 if they are equal, and negative if left is smaller than right. */ + @Override public int compareTo(Object right_obj) { return compareTo(((Value) right_obj).get()); } @@ -182,7 +180,6 @@ public class Value implements WritableCo return (diff != 0) ? diff : WritableComparator.compareBytes(this.value, 0, this.value.length, that, 0, that.length); } - /** {@inheritDoc} */ @Override public boolean equals(Object right_obj) { if (right_obj instanceof byte[]) { @@ -196,13 +193,7 @@ public class Value implements WritableCo @Override public String toString() { - String retValue = ""; - try { - retValue = new String(get(), Constants.VALUE_ENCODING); - } catch (UnsupportedEncodingException e) { - log.error(e.toString()); - } - return retValue; + return new String(get(), Constants.UTF8); } /** @@ -216,7 +207,6 @@ public class Value implements WritableCo super(Value.class); } - /** {@inheritDoc} */ @Override public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) { return comparator.compare(b1, s1, l1, b2, s2, l2); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/MultiScanResult.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/MultiScanResult.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/MultiScanResult.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/MultiScanResult.java Mon Jun 24 21:34:20 2013 @@ -854,7 +854,7 @@ import org.slf4j.LoggerFactory; struct.results = new ArrayList(_list16.size); for (int _i17 = 0; _i17 < _list16.size; ++_i17) { - TKeyValue _elem18; // required + TKeyValue _elem18; // optional _elem18 = new TKeyValue(); _elem18.read(iprot); struct.results.add(_elem18); @@ -882,7 +882,7 @@ import org.slf4j.LoggerFactory; _val22 = new ArrayList(_list23.size); for (int _i24 = 0; _i24 < _list23.size; ++_i24) { - TRange _elem25; // required + TRange _elem25; // optional _elem25 = new TRange(); _elem25.read(iprot); _val22.add(_elem25); @@ -905,7 +905,7 @@ import org.slf4j.LoggerFactory; struct.fullScans = new ArrayList(_list26.size); for (int _i27 = 0; _i27 < _list26.size; ++_i27) { - TKeyExtent _elem28; // required + TKeyExtent _elem28; // optional _elem28 = new TKeyExtent(); _elem28.read(iprot); struct.fullScans.add(_elem28); @@ -1124,7 +1124,7 @@ import org.slf4j.LoggerFactory; struct.results = new ArrayList(_list37.size); for (int _i38 = 0; _i38 < _list37.size; ++_i38) { - TKeyValue _elem39; // required + TKeyValue _elem39; // optional _elem39 = new TKeyValue(); _elem39.read(iprot); struct.results.add(_elem39); @@ -1147,7 +1147,7 @@ import org.slf4j.LoggerFactory; _val43 = new ArrayList(_list44.size); for (int _i45 = 0; _i45 < _list44.size; ++_i45) { - TRange _elem46; // required + TRange _elem46; // optional _elem46 = new TRange(); _elem46.read(iprot); _val43.add(_elem46); @@ -1164,7 +1164,7 @@ import org.slf4j.LoggerFactory; struct.fullScans = new ArrayList(_list47.size); for (int _i48 = 0; _i48 < _list47.size; ++_i48) { - TKeyExtent _elem49; // required + TKeyExtent _elem49; // optional _elem49 = new TKeyExtent(); _elem49.read(iprot); struct.fullScans.add(_elem49); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/ScanResult.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/ScanResult.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/ScanResult.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/ScanResult.java Mon Jun 24 21:34:20 2013 @@ -436,7 +436,7 @@ import org.slf4j.LoggerFactory; struct.results = new ArrayList(_list8.size); for (int _i9 = 0; _i9 < _list8.size; ++_i9) { - TKeyValue _elem10; // required + TKeyValue _elem10; // optional _elem10 = new TKeyValue(); _elem10.read(iprot); struct.results.add(_elem10); @@ -535,7 +535,7 @@ import org.slf4j.LoggerFactory; struct.results = new ArrayList(_list13.size); for (int _i14 = 0; _i14 < _list13.size; ++_i14) { - TKeyValue _elem15; // required + TKeyValue _elem15; // optional _elem15 = new TKeyValue(); _elem15.read(iprot); struct.results.add(_elem15); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/TMutation.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/TMutation.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/TMutation.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/TMutation.java Mon Jun 24 21:34:20 2013 @@ -630,7 +630,7 @@ import org.slf4j.LoggerFactory; struct.values = new ArrayList(_list0.size); for (int _i1 = 0; _i1 < _list0.size; ++_i1) { - ByteBuffer _elem2; // required + ByteBuffer _elem2; // optional _elem2 = iprot.readBinary(); struct.values.add(_elem2); } @@ -758,7 +758,7 @@ import org.slf4j.LoggerFactory; struct.values = new ArrayList(_list5.size); for (int _i6 = 0; _i6 < _list5.size; ++_i6) { - ByteBuffer _elem7; // required + ByteBuffer _elem7; // optional _elem7 = iprot.readBinary(); struct.values.add(_elem7); } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/UpdateErrors.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/UpdateErrors.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/UpdateErrors.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/data/thrift/UpdateErrors.java Mon Jun 24 21:34:20 2013 @@ -584,7 +584,7 @@ import org.slf4j.LoggerFactory; struct.violationSummaries = new ArrayList(_list54.size); for (int _i55 = 0; _i55 < _list54.size; ++_i55) { - TConstraintViolationSummary _elem56; // required + TConstraintViolationSummary _elem56; // optional _elem56 = new TConstraintViolationSummary(); _elem56.read(iprot); struct.violationSummaries.add(_elem56); @@ -755,7 +755,7 @@ import org.slf4j.LoggerFactory; struct.violationSummaries = new ArrayList(_list71.size); for (int _i72 = 0; _i72 < _list71.size; ++_i72) { - TConstraintViolationSummary _elem73; // required + TConstraintViolationSummary _elem73; // optional _elem73 = new TConstraintViolationSummary(); _elem73.read(iprot); struct.violationSummaries.add(_elem73); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/file/FileOperations.java Mon Jun 24 21:34:20 2013 @@ -44,14 +44,13 @@ class DispatchingFileFactory extends Fil if (name.startsWith(Constants.MAPFILE_EXTENSION + "_")) { return new MapFileOperations(); } - String[] sp = name.split("\\."); - if (sp.length != 2) { + if (sp.length < 2) { throw new IllegalArgumentException("File name " + name + " has no extension"); } - String extension = sp[1]; + String extension = sp[sp.length - 1]; if (extension.equals(Constants.MAPFILE_EXTENSION) || extension.equals(Constants.MAPFILE_EXTENSION + "_tmp")) { return new MapFileOperations(); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/iterators/OptionDescriber.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/iterators/OptionDescriber.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/iterators/OptionDescriber.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/iterators/OptionDescriber.java Mon Jun 24 21:34:20 2013 @@ -51,7 +51,7 @@ public interface OptionDescriber { * is a list of descriptions of additional options that don't have fixed names (null if unused). The descriptions are intended to describe a * category, and the user will provide parameter names and values in that category; e.g., the FilteringIterator needs a list of Filters intended to * be named by their priority numbers, so its unnamedOptionDescriptions = - * Collections.singletonList(" ") + * Collections.singletonList("<filterPriorityNumber> <ageoff|regex|filterClass>") */ public IteratorOptions(String name, String description, Map namedOptions, List unnamedOptionDescriptions) { this.name = name; Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterClientService.java Mon Jun 24 21:34:20 2013 @@ -15517,7 +15517,7 @@ import org.slf4j.LoggerFactory; struct.arguments = new ArrayList(_list70.size); for (int _i71 = 0; _i71 < _list70.size; ++_i71) { - ByteBuffer _elem72; // required + ByteBuffer _elem72; // optional _elem72 = iprot.readBinary(); struct.arguments.add(_elem72); } @@ -15721,7 +15721,7 @@ import org.slf4j.LoggerFactory; struct.arguments = new ArrayList(_list81.size); for (int _i82 = 0; _i82 < _list81.size; ++_i82) { - ByteBuffer _elem83; // required + ByteBuffer _elem83; // optional _elem83 = iprot.readBinary(); struct.arguments.add(_elem83); } Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterMonitorInfo.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterMonitorInfo.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterMonitorInfo.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/MasterMonitorInfo.java Mon Jun 24 21:34:20 2013 @@ -1041,7 +1041,7 @@ import org.slf4j.LoggerFactory; struct.tServerInfo = new ArrayList(_list22.size); for (int _i23 = 0; _i23 < _list22.size; ++_i23) { - TabletServerStatus _elem24; // required + TabletServerStatus _elem24; // optional _elem24 = new TabletServerStatus(); _elem24.read(iprot); struct.tServerInfo.add(_elem24); @@ -1104,7 +1104,7 @@ import org.slf4j.LoggerFactory; struct.serversShuttingDown = new HashSet(2*_set29.size); for (int _i30 = 0; _i30 < _set29.size; ++_i30) { - String _elem31; // required + String _elem31; // optional _elem31 = iprot.readString(); struct.serversShuttingDown.add(_elem31); } @@ -1122,7 +1122,7 @@ import org.slf4j.LoggerFactory; struct.deadTabletServers = new ArrayList(_list32.size); for (int _i33 = 0; _i33 < _list32.size; ++_i33) { - DeadServer _elem34; // required + DeadServer _elem34; // optional _elem34 = new DeadServer(); _elem34.read(iprot); struct.deadTabletServers.add(_elem34); @@ -1351,7 +1351,7 @@ import org.slf4j.LoggerFactory; struct.tServerInfo = new ArrayList(_list49.size); for (int _i50 = 0; _i50 < _list49.size; ++_i50) { - TabletServerStatus _elem51; // required + TabletServerStatus _elem51; // optional _elem51 = new TabletServerStatus(); _elem51.read(iprot); struct.tServerInfo.add(_elem51); @@ -1392,7 +1392,7 @@ import org.slf4j.LoggerFactory; struct.serversShuttingDown = new HashSet(2*_set56.size); for (int _i57 = 0; _i57 < _set56.size; ++_i57) { - String _elem58; // required + String _elem58; // optional _elem58 = iprot.readString(); struct.serversShuttingDown.add(_elem58); } @@ -1405,7 +1405,7 @@ import org.slf4j.LoggerFactory; struct.deadTabletServers = new ArrayList(_list59.size); for (int _i60 = 0; _i60 < _list59.size; ++_i60) { - DeadServer _elem61; // required + DeadServer _elem61; // optional _elem61 = new DeadServer(); _elem61.read(iprot); struct.deadTabletServers.add(_elem61); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/TabletServerStatus.java Mon Jun 24 21:34:20 2013 @@ -1219,7 +1219,7 @@ import org.slf4j.LoggerFactory; struct.logSorts = new ArrayList(_list4.size); for (int _i5 = 0; _i5 < _list4.size; ++_i5) { - RecoveryStatus _elem6; // required + RecoveryStatus _elem6; // optional _elem6 = new RecoveryStatus(); _elem6.read(iprot); struct.logSorts.add(_elem6); @@ -1462,7 +1462,7 @@ import org.slf4j.LoggerFactory; struct.logSorts = new ArrayList(_list15.size); for (int _i16 = 0; _i16 < _list15.size; ++_i16) { - RecoveryStatus _elem17; // required + RecoveryStatus _elem17; // optional _elem17 = new RecoveryStatus(); _elem17.read(iprot); struct.logSorts.add(_elem17); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/TabletSplit.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/TabletSplit.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/TabletSplit.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/master/thrift/TabletSplit.java Mon Jun 24 21:34:20 2013 @@ -448,7 +448,7 @@ import org.slf4j.LoggerFactory; struct.newTablets = new ArrayList(_list62.size); for (int _i63 = 0; _i63 < _list62.size; ++_i63) { - org.apache.accumulo.core.data.thrift.TKeyExtent _elem64; // required + org.apache.accumulo.core.data.thrift.TKeyExtent _elem64; // optional _elem64 = new org.apache.accumulo.core.data.thrift.TKeyExtent(); _elem64.read(iprot); struct.newTablets.add(_elem64); @@ -546,7 +546,7 @@ import org.slf4j.LoggerFactory; struct.newTablets = new ArrayList(_list67.size); for (int _i68 = 0; _i68 < _list67.size; ++_i68) { - org.apache.accumulo.core.data.thrift.TKeyExtent _elem69; // required + org.apache.accumulo.core.data.thrift.TKeyExtent _elem69; // optional _elem69 = new org.apache.accumulo.core.data.thrift.TKeyExtent(); _elem69.read(iprot); struct.newTablets.add(_elem69); Modified: accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java URL: http://svn.apache.org/viewvc/accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java?rev=1496226&r1=1496225&r2=1496226&view=diff ============================================================================== --- accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java (original) +++ accumulo/branches/ACCUMULO-CURATOR/core/src/main/java/org/apache/accumulo/core/security/Authorizations.java Mon Jun 24 21:34:20 2013 @@ -41,6 +41,8 @@ public class Authorizations implements I private Set auths = new HashSet(); private List authsList = new ArrayList(); + public static final Authorizations EMPTY = new Authorizations(); + private static final boolean[] validAuthChars = new boolean[256]; public static final String HEADER = "!AUTH1:";