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 2B60D105A2 for ; Tue, 4 Feb 2014 17:55:37 +0000 (UTC) Received: (qmail 27218 invoked by uid 500); 4 Feb 2014 17:55:16 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 26770 invoked by uid 500); 4 Feb 2014 17:55:03 -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 25446 invoked by uid 99); 4 Feb 2014 17:54:34 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 04 Feb 2014 17:54:34 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 84D2C824173; Tue, 4 Feb 2014 17:54:31 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: elserj@apache.org To: commits@accumulo.apache.org Date: Tue, 04 Feb 2014 17:55:16 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [47/48] git commit: Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT Merge branch '1.5.1-SNAPSHOT' into 1.6.0-SNAPSHOT Conflicts: core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java core/src/main/java/org/apache/accumulo/core/client/mapred/InputFormatBase.java core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java core/src/main/java/org/apache/accumulo/core/util/Merge.java core/src/main/java/org/apache/accumulo/core/util/MetadataTable.java core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java core/src/main/java/org/apache/accumulo/core/util/shell/commands/HelpCommand.java core/src/main/java/org/apache/accumulo/core/util/shell/commands/HiddenCommand.java core/src/main/java/org/apache/accumulo/core/util/shell/commands/HistoryCommand.java core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooLock.java fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooReservation.java minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java server/base/src/main/java/org/apache/accumulo/server/constraints/MetadataConstraints.java server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java server/base/src/main/java/org/apache/accumulo/server/master/LiveTServerSet.java server/base/src/main/java/org/apache/accumulo/server/master/state/DeadServerList.java server/base/src/main/java/org/apache/accumulo/server/master/state/MetaDataTableScanner.java server/base/src/main/java/org/apache/accumulo/server/master/state/TServerInstance.java server/base/src/main/java/org/apache/accumulo/server/master/state/ZooTabletStateStore.java server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKAuthorizor.java server/base/src/main/java/org/apache/accumulo/server/security/handler/ZKPermHandler.java server/base/src/main/java/org/apache/accumulo/server/tables/TableManager.java server/base/src/main/java/org/apache/accumulo/server/util/AddFilesWithMissingEntries.java server/base/src/main/java/org/apache/accumulo/server/util/Info.java server/base/src/main/java/org/apache/accumulo/server/util/LocalityCheck.java server/base/src/main/java/org/apache/accumulo/server/util/SendLogToChainsaw.java server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java server/master/src/main/java/org/apache/accumulo/master/tableOps/CancelCompactions.java server/master/src/main/java/org/apache/accumulo/master/tableOps/CompactRange.java server/master/src/main/java/org/apache/accumulo/master/tableOps/ExportTable.java server/master/src/main/java/org/apache/accumulo/master/tableOps/ImportTable.java server/master/src/main/java/org/apache/accumulo/master/tableOps/RenameTable.java server/master/src/main/java/org/apache/accumulo/master/tableOps/Utils.java server/monitor/src/main/java/org/apache/accumulo/monitor/EmbeddedWebServer.java server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/DefaultServlet.java server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/ShellServlet.java server/monitor/src/main/java/org/apache/accumulo/monitor/servlets/TServersServlet.java server/src/main/java/org/apache/accumulo/server/gc/SimpleGarbageCollector.java server/src/main/java/org/apache/accumulo/server/master/Master.java server/src/main/java/org/apache/accumulo/server/security/SecurityConstants.java server/src/main/java/org/apache/accumulo/server/util/MetadataTable.java server/src/main/java/org/apache/accumulo/server/util/RemoveEntriesForMissingFiles.java server/tserver/src/main/java/org/apache/accumulo/tserver/Compactor.java server/tserver/src/main/java/org/apache/accumulo/tserver/Tablet.java server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java server/tserver/src/main/java/org/apache/accumulo/tserver/log/DfsLogger.java start/src/main/java/org/apache/accumulo/start/classloader/vfs/PostDelegatingVFSClassLoader.java test/src/main/java/org/apache/accumulo/test/TestBinaryRows.java test/src/main/java/org/apache/accumulo/test/TestMultiTableIngest.java test/src/main/java/org/apache/accumulo/test/VerifyIngest.java test/src/main/java/org/apache/accumulo/test/functional/BloomFilterTest.java test/src/main/java/org/apache/accumulo/test/functional/BulkSplitOptimizationTest.java test/src/main/java/org/apache/accumulo/test/functional/FunctionalTest.java test/src/main/java/org/apache/accumulo/test/functional/PermissionsTest.java test/src/main/java/org/apache/accumulo/test/functional/RunTests.java test/src/main/java/org/apache/accumulo/test/performance/metadata/MetadataBatchScanTest.java test/src/main/java/org/apache/accumulo/test/randomwalk/concurrent/StopTabletServer.java test/src/test/java/org/apache/accumulo/test/functional/BadIteratorMincIT.java test/src/test/java/org/apache/accumulo/test/functional/ConstraintIT.java test/src/test/java/org/apache/accumulo/test/functional/CreateAndUseIT.java test/src/test/java/org/apache/accumulo/test/functional/NativeMapIT.java test/src/test/java/org/apache/accumulo/test/functional/ScanIteratorIT.java test/src/test/java/org/apache/accumulo/test/functional/ServerSideErrorIT.java test/src/test/java/org/apache/accumulo/test/functional/TabletIT.java Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/7688eaf0 Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/7688eaf0 Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/7688eaf0 Branch: refs/heads/master Commit: 7688eaf0b684a2bb76a0b49bff9a79763d41a8e5 Parents: 2a8e146 b11a036 Author: Josh Elser Authored: Tue Feb 4 12:19:38 2014 -0500 Committer: Josh Elser Committed: Tue Feb 4 12:19:38 2014 -0500 ---------------------------------------------------------------------- .../apache/accumulo/core/cli/ClientOpts.java | 2 +- .../accumulo/core/client/ZooKeeperInstance.java | 8 ++-- .../client/admin/InstanceOperationsImpl.java | 2 +- .../core/client/admin/TableOperationsImpl.java | 28 ++++++------ .../accumulo/core/client/impl/ServerClient.java | 2 +- .../accumulo/core/client/impl/Tables.java | 2 +- .../accumulo/core/client/impl/Writer.java | 3 +- .../core/client/mapreduce/RangeInputSplit.java | 4 +- .../lib/partition/RangePartitioner.java | 7 ++- .../mapreduce/lib/util/InputConfigurator.java | 4 +- .../accumulo/core/client/mock/MockShell.java | 5 +- .../core/conf/DefaultConfiguration.java | 8 ++-- .../accumulo/core/data/ArrayByteSequence.java | 6 ++- .../org/apache/accumulo/core/data/Column.java | 5 +- .../org/apache/accumulo/core/data/KeyValue.java | 4 +- .../accumulo/core/file/BloomFilterLayer.java | 5 +- .../core/file/rfile/bcfile/CompareUtils.java | 9 +++- .../core/iterators/ColumnFamilyCounter.java | 3 +- .../accumulo/core/iterators/IteratorUtil.java | 6 ++- .../accumulo/core/iterators/LongCombiner.java | 5 +- .../accumulo/core/iterators/conf/ColumnSet.java | 5 +- .../core/iterators/user/BigDecimalCombiner.java | 5 +- .../core/iterators/user/GrepIterator.java | 3 +- .../iterators/user/IntersectingIterator.java | 11 +++-- .../core/iterators/user/LargeRowFilter.java | 3 +- .../core/iterators/user/RegExFilter.java | 2 +- .../iterators/user/RowDeletingIterator.java | 3 +- .../iterators/user/SummingArrayCombiner.java | 5 +- .../iterators/user/TransformingIterator.java | 16 ++++--- .../core/security/ColumnVisibility.java | 40 ++++++++-------- .../core/security/VisibilityConstraint.java | 5 +- .../core/security/VisibilityParseException.java | 4 +- .../core/trace/InstanceUserPassword.java | 3 +- .../accumulo/core/trace/ZooTraceClient.java | 5 +- .../core/util/ByteArrayBackedCharSequence.java | 3 +- .../apache/accumulo/core/util/ByteArraySet.java | 4 +- .../accumulo/core/util/ByteBufferUtil.java | 3 +- .../org/apache/accumulo/core/util/Encoding.java | 5 +- .../apache/accumulo/core/util/FastFormat.java | 4 +- .../org/apache/accumulo/core/util/Merge.java | 3 +- .../org/apache/accumulo/core/util/TextUtil.java | 2 +- .../apache/accumulo/core/util/shell/Shell.java | 9 ++-- .../util/shell/commands/AddSplitsCommand.java | 5 +- .../shell/commands/AuthenticateCommand.java | 3 +- .../util/shell/commands/CreateTableCommand.java | 6 +-- .../util/shell/commands/ExecfileCommand.java | 3 +- .../util/shell/commands/GetSplitsCommand.java | 5 +- .../core/util/shell/commands/HiddenCommand.java | 3 +- .../util/shell/commands/ListIterCommand.java | 6 ++- .../core/util/shell/commands/PasswdCommand.java | 3 +- .../shell/commands/QuotedStringTokenizer.java | 3 +- .../util/shell/commands/SetIterCommand.java | 2 - .../core/util/shell/commands/UserCommand.java | 3 +- .../shell/commands/UserPermissionsCommand.java | 18 ++++---- .../org/apache/accumulo/fate/AdminUtil.java | 6 ++- .../org/apache/accumulo/fate/AgeOffStore.java | 1 + .../java/org/apache/accumulo/fate/ZooStore.java | 14 +++--- .../zookeeper/DistributedReadWriteLock.java | 16 ++++--- .../accumulo/fate/zookeeper/ZooCache.java | 6 ++- .../apache/accumulo/fate/zookeeper/ZooLock.java | 7 +-- .../accumulo/fate/zookeeper/ZooReservation.java | 12 +++-- .../accumulo/fate/zookeeper/ZooSession.java | 5 +- .../org/apache/accumulo/proxy/ProxyServer.java | 8 ++-- .../apache/accumulo/proxy/TestProxyClient.java | 13 +++--- .../java/org/apache/accumulo/proxy/Util.java | 9 ++-- .../org/apache/accumulo/server/Accumulo.java | 4 +- .../accumulo/server/client/HdfsZooInstance.java | 8 ++-- .../server/conf/TableConfiguration.java | 2 +- .../accumulo/server/conf/ZooConfiguration.java | 2 +- .../server/constraints/MetadataConstraints.java | 14 +++--- .../apache/accumulo/server/init/Initialize.java | 48 ++++++++++---------- .../accumulo/server/master/LiveTServerSet.java | 2 +- .../server/master/state/DeadServerList.java | 5 +- .../master/state/MetaDataTableScanner.java | 2 +- .../server/master/state/TServerInstance.java | 5 +- .../master/state/TabletStateChangeIterator.java | 5 +- .../accumulo/server/master/state/ZooStore.java | 3 +- .../master/state/ZooTabletStateStore.java | 9 ++-- .../server/metrics/AbstractMetricsImpl.java | 12 +++-- .../accumulo/server/monitor/LogService.java | 2 +- .../server/security/SecurityOperation.java | 4 +- .../security/handler/ZKAuthenticator.java | 2 +- .../server/security/handler/ZKAuthorizor.java | 3 +- .../server/security/handler/ZKPermHandler.java | 3 +- .../accumulo/server/tables/TableManager.java | 17 +++---- .../server/tablets/UniqueNameAllocator.java | 8 ++-- .../server/util/AddFilesWithMissingEntries.java | 3 +- .../accumulo/server/util/ChangeSecret.java | 3 +- .../accumulo/server/util/CleanZookeeper.java | 4 +- .../accumulo/server/util/DeleteZooInstance.java | 4 +- .../accumulo/server/util/ListInstances.java | 4 +- .../accumulo/server/util/LocalityCheck.java | 6 ++- .../accumulo/server/util/SendLogToChainsaw.java | 14 +++--- .../accumulo/server/util/SystemPropUtil.java | 2 +- .../accumulo/server/util/TabletServerLocks.java | 2 +- .../server/zookeeper/DistributedWorkQueue.java | 2 +- .../accumulo/server/zookeeper/ZooQueueLock.java | 7 +-- .../server/zookeeper/ZooReaderWriter.java | 3 +- .../accumulo/utils/metanalysis/IndexMeta.java | 7 +-- .../master/recovery/RecoveryManager.java | 2 +- .../accumulo/master/state/MergeStats.java | 6 +-- .../accumulo/master/state/SetGoalState.java | 2 +- .../accumulo/master/tableOps/BulkImport.java | 6 +-- .../master/tableOps/CancelCompactions.java | 11 ++--- .../accumulo/master/tableOps/CompactRange.java | 16 +++---- .../accumulo/master/tableOps/ExportTable.java | 6 +-- .../accumulo/master/tableOps/ImportTable.java | 10 ++-- .../accumulo/master/tableOps/RenameTable.java | 4 +- .../apache/accumulo/master/tableOps/Utils.java | 19 ++++---- .../master/tserverOps/ShutdownTServer.java | 2 +- .../apache/accumulo/master/util/FateAdmin.java | 8 +++- .../accumulo/monitor/EmbeddedWebServer.java | 9 ++-- .../org/apache/accumulo/monitor/Monitor.java | 4 +- .../accumulo/monitor/ZooKeeperStatus.java | 5 +- .../monitor/servlets/DefaultServlet.java | 3 +- .../accumulo/monitor/servlets/LogServlet.java | 6 +-- .../accumulo/monitor/servlets/ShellServlet.java | 4 +- .../monitor/servlets/TServersServlet.java | 3 +- .../accumulo/monitor/servlets/trace/Basic.java | 3 +- .../org/apache/accumulo/tracer/TraceServer.java | 6 +-- .../tserver/BulkFailedCopyProcessor.java | 3 +- .../org/apache/accumulo/tserver/Compactor.java | 2 +- .../apache/accumulo/tserver/InMemoryMap.java | 5 +- .../org/apache/accumulo/tserver/Tablet.java | 17 ++++--- .../apache/accumulo/tserver/TabletServer.java | 10 ++-- .../apache/accumulo/tserver/log/DfsLogger.java | 3 +- .../accumulo/tserver/logger/LogFileValue.java | 12 +++-- .../accumulo/tserver/logger/LogReader.java | 3 +- .../metrics/TabletServerScanMetrics.java | 2 +- .../start/classloader/AccumuloClassLoader.java | 6 +-- .../vfs/providers/HdfsRandomAccessContent.java | 3 +- .../accumulo/test/BulkImportDirectory.java | 5 +- .../accumulo/test/NativeMapConcurrencyTest.java | 3 +- .../accumulo/test/NativeMapPerformanceTest.java | 7 +-- .../accumulo/test/NativeMapStressTest.java | 5 +- .../apache/accumulo/test/TestBinaryRows.java | 25 +++++----- .../org/apache/accumulo/test/TestIngest.java | 5 +- .../accumulo/test/TestMultiTableIngest.java | 3 +- .../org/apache/accumulo/test/VerifyIngest.java | 1 + .../test/continuous/ContinuousIngest.java | 6 +-- .../test/continuous/ContinuousMoru.java | 5 +- .../test/continuous/ContinuousQuery.java | 3 +- .../test/continuous/ContinuousScanner.java | 3 +- .../test/continuous/ContinuousVerify.java | 5 +- .../test/continuous/ContinuousWalk.java | 15 +++--- .../accumulo/test/continuous/Histogram.java | 4 +- .../test/continuous/PrintScanTimeHistogram.java | 4 +- .../accumulo/test/continuous/TimeBinner.java | 3 +- .../test/continuous/UndefinedAnalyzer.java | 18 ++++---- .../test/functional/CacheTestReader.java | 5 +- .../test/functional/CacheTestWriter.java | 9 ++-- .../accumulo/test/functional/ZombieTServer.java | 2 +- .../metadata/MetadataBatchScanTest.java | 5 +- .../apache/accumulo/test/randomwalk/Module.java | 5 +- .../test/randomwalk/bulk/BulkMinusOne.java | 3 +- .../test/randomwalk/bulk/BulkPlusOne.java | 3 +- .../accumulo/test/randomwalk/bulk/Verify.java | 5 +- .../test/randomwalk/concurrent/BatchWrite.java | 3 +- .../test/randomwalk/concurrent/BulkImport.java | 3 +- .../concurrent/ChangeAuthorizations.java | 5 +- .../randomwalk/concurrent/StopTabletServer.java | 2 +- .../accumulo/test/randomwalk/image/Verify.java | 7 +-- .../accumulo/test/randomwalk/image/Write.java | 7 +-- .../test/randomwalk/multitable/CopyTable.java | 2 +- .../test/randomwalk/multitable/CreateTable.java | 2 +- .../test/randomwalk/multitable/Write.java | 3 +- .../randomwalk/security/SecurityHelper.java | 7 +-- .../test/randomwalk/security/TableOp.java | 7 +-- .../randomwalk/security/WalkingSecurity.java | 2 +- .../randomwalk/sequential/MapRedVerifyTool.java | 2 +- .../test/randomwalk/sequential/Write.java | 3 +- .../test/randomwalk/shard/BulkInsert.java | 5 +- .../test/randomwalk/shard/ExportIndex.java | 3 +- .../test/randomwalk/shard/ShardFixture.java | 2 +- .../accumulo/test/scalability/Ingest.java | 3 +- .../accumulo/test/functional/AddSplitIT.java | 3 +- .../test/functional/BadIteratorMincIT.java | 6 ++- .../test/functional/BatchScanSplitIT.java | 5 +- .../test/functional/BatchWriterFlushIT.java | 3 +- .../accumulo/test/functional/BulkFileIT.java | 3 +- .../accumulo/test/functional/ConcurrencyIT.java | 5 +- .../accumulo/test/functional/ConstraintIT.java | 21 +++++---- .../test/functional/CreateAndUseIT.java | 3 +- .../test/functional/DeleteEverythingIT.java | 3 +- .../test/functional/DeleteRowsSplitIT.java | 5 +- .../accumulo/test/functional/LargeRowIT.java | 5 +- .../accumulo/test/functional/NativeMapIT.java | 24 +++++----- .../test/functional/ScanIteratorIT.java | 5 +- .../accumulo/test/functional/ScanRangeIT.java | 3 +- .../test/functional/ScanSessionTimeOutIT.java | 3 +- .../test/functional/ServerSideErrorIT.java | 2 +- .../test/functional/SplitRecoveryIT.java | 4 +- .../accumulo/test/functional/TabletIT.java | 5 +- .../accumulo/test/functional/VisibilityIT.java | 9 ++-- .../trace/instrument/TraceRunnable.java | 9 ++++ .../instrument/receivers/ZooSpanClient.java | 4 +- .../trace/instrument/CountSamplerTest.java | 2 - .../trace/instrument/PerformanceTest.java | 2 - .../accumulo/trace/instrument/TracerTest.java | 3 -- 199 files changed, 691 insertions(+), 511 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java index 4003e6f,448981b..f459275 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java @@@ -212,19 -200,15 +212,19 @@@ public class TableOperationsImpl extend public void create(String tableName, boolean limitVersion, TimeType timeType) throws AccumuloException, AccumuloSecurityException, TableExistsException { ArgumentChecker.notNull(tableName, timeType); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(timeType.name().getBytes())); + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(timeType.name().getBytes(Constants.UTF8))); - Map opts = IteratorUtil.generateInitialTableProperties(limitVersion); + Map opts; + if (limitVersion) + opts = IteratorUtil.generateInitialTableProperties(limitVersion); + else + opts = Collections.emptyMap(); try { - doTableOperation(TableOperation.CREATE, args, opts); - } catch (TableNotFoundException e1) { + doTableFateOperation(tableName, AccumuloException.class, FateOperation.TABLE_CREATE, args, opts); + } catch (TableNotFoundException e) { // should not happen - throw new RuntimeException(e1); + throw new AssertionError(e); } } @@@ -698,7 -671,10 +698,7 @@@ if (propertiesToSet == null) propertiesToSet = Collections.emptyMap(); - List args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes()), ByteBuffer.wrap(newTableName.getBytes())); - if (!Collections.disjoint(propertiesToExclude, propertiesToSet.keySet())) - throw new IllegalArgumentException("propertiesToSet and propertiesToExclude not disjoint"); - + List args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8))); Map opts = new HashMap(); for (Entry entry : propertiesToSet.entrySet()) { if (entry.getKey().startsWith(CLONE_EXCLUDE_PREFIX)) @@@ -1168,14 -1095,22 +1168,14 @@@ public void importDirectory(String tableName, String dir, String failureDir, boolean setTime) throws IOException, AccumuloSecurityException, TableNotFoundException, AccumuloException { ArgumentChecker.notNull(tableName, dir, failureDir); - FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), instance.getConfiguration()); - Path dirPath = fs.makeQualified(new Path(dir)); - Path failPath = fs.makeQualified(new Path(failureDir)); - if (!fs.exists(dirPath)) - throw new AccumuloException("Bulk import directory " + dir + " does not exist!"); - if (!fs.exists(failPath)) - throw new AccumuloException("Bulk import failure directory " + failureDir + " does not exist!"); - FileStatus[] listStatus = fs.listStatus(failPath); - if (listStatus != null && listStatus.length != 0) { - if (listStatus.length == 1 && listStatus[0].isDir()) - throw new AccumuloException("Bulk import directory " + failPath + " is a file"); - throw new AccumuloException("Bulk import failure directory " + failPath + " is not empty"); - } + // check for table existance + Tables.getTableId(instance, tableName); + + Path dirPath = checkPath(dir, "Bulk", ""); + Path failPath = checkPath(failureDir, "Bulk", "failure"); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(dirPath.toString().getBytes()), - ByteBuffer.wrap(failPath.toString().getBytes()), ByteBuffer.wrap((setTime + "").getBytes())); + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(dirPath.toString().getBytes(Constants.UTF8)), + ByteBuffer.wrap(failPath.toString().getBytes(Constants.UTF8)), ByteBuffer.wrap((setTime + "").getBytes(Constants.UTF8))); Map opts = new HashMap(); try { @@@ -1319,11 -1134,10 +1319,11 @@@ * @throws TableNotFoundException */ @Override - public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { + public void offline(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { ArgumentChecker.notNull(tableName); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8))); + String tableId = Tables.getTableId(instance, tableName); - List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes())); ++ List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8))); Map opts = new HashMap(); try { @@@ -1353,10 -1159,9 +1353,10 @@@ * @throws TableNotFoundException */ @Override - public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { + public void online(String tableName, boolean wait) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { ArgumentChecker.notNull(tableName); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8))); + String tableId = Tables.getTableId(instance, tableName); - List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes())); ++ List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8))); Map opts = new HashMap(); try { @@@ -1455,15 -1213,11 +1455,15 @@@ ZipEntry zipEntry; while ((zipEntry = zis.getNextEntry()) != null) { if (zipEntry.getName().equals(Constants.EXPORT_TABLE_CONFIG_FILE)) { - BufferedReader in = new BufferedReader(new InputStreamReader(zis)); + BufferedReader in = new BufferedReader(new InputStreamReader(zis, Constants.UTF8)); - String line; - while ((line = in.readLine()) != null) { - String sa[] = line.split("=", 2); - props.put(sa[0], sa[1]); + try { + String line; + while ((line = in.readLine()) != null) { + String sa[] = line.split("=", 2); + props.put(sa[0], sa[1]); + } + } finally { + in.close(); } break; http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java index 4eb845d,5b97941..bf3e1bc --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java @@@ -139,10 -139,14 +139,10 @@@ public class ServerClient for (String tserver : zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTSERVERS)) { String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + tserver; byte[] data = ZooUtil.getLockData(zc, path); - if (data != null && !new String(data).equals("master")) - if (data != null) { - String tserverData = new String(data, Constants.UTF8); - if (!tserverData.equals("master")) { - servers.add(new ThriftTransportKey(new ServerServices(tserverData).getAddressString(Service.TSERV_CLIENT), conf.getPort(Property.TSERV_CLIENTPORT), - rpcTimeout)); - } - } ++ if (data != null && !new String(data, Constants.UTF8).equals("master")) + servers.add(new ThriftTransportKey( + new ServerServices(new String(data)).getAddressString(Service.TSERV_CLIENT), + rpcTimeout, SslConnectionParams.forClient(ServerConfigurationUtil.getConfiguration(instance)))); } boolean opened = false; http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java index 8a2c2c7,07f75e0..404aa32 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java @@@ -249,10 -111,10 +249,10 @@@ public class Tables byte[] state = zc.get(statePath); if (state == null) return TableState.UNKNOWN; - + - return TableState.valueOf(new String(state)); + return TableState.valueOf(new String(state, Constants.UTF8)); } - + public static long getCacheResetCount() { return cacheResetCount.get(); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java index 72a050a,7353e54..950eb4d --- a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java @@@ -16,6 -16,8 +16,7 @@@ */ package org.apache.accumulo.core.client.impl; -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; @@@ -84,10 -85,10 +85,10 @@@ public class Writer throw new IllegalArgumentException("Can not add empty mutations"); while (true) { - TabletLocation tabLoc = TabletLocator.getInstance(instance, table).locateTablet(new Text(m.getRow()), false, true, credentials); + TabletLocation tabLoc = TabletLocator.getLocator(instance, table).locateTablet(credentials, new Text(m.getRow()), false, true); if (tabLoc == null) { - log.trace("No tablet location found for row " + new String(m.getRow())); + log.trace("No tablet location found for row " + new String(m.getRow(), Constants.UTF8)); UtilWaitThread.sleep(500); continue; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java index b238903,73c9b59..85c22fa --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java @@@ -68,22 -63,11 +68,22 @@@ public class RangeInputSplit extends In public RangeInputSplit() { range = new Range(); locations = new String[0]; + tableName = ""; + tableId = ""; } - public RangeInputSplit(Range range, String[] locations) { + public RangeInputSplit(RangeInputSplit split) throws IOException { + this.setRange(split.getRange()); + this.setLocations(split.getLocations()); + this.setTableName(split.getTableName()); + this.setTableId(split.getTableId()); + } + + protected RangeInputSplit(String table, String tableId, Range range, String[] locations) { this.range = range; - this.locations = locations; + setLocations(locations); + this.tableName = table; + this.tableId = tableId; } public Range getRange() { http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java index bdea610,e29c791..3cb69ad --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java @@@ -25,7 -27,7 +27,8 @@@ import java.util.Arrays import java.util.Scanner; import java.util.TreeSet; + import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.client.mapreduce.lib.util.DistributedCacheHelper; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java index 270c5a1,d9e313c..c76a185 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java @@@ -152,11 -138,11 +152,11 @@@ public class InputConfigurator extends */ public static Authorizations getScanAuthorizations(Class implementingClass, Configuration conf) { String authString = conf.get(enumToConfKey(implementingClass, ScanOpts.AUTHORIZATIONS)); - return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes()); - return authString == null ? Constants.NO_AUTHS : new Authorizations(authString.getBytes(Constants.UTF8)); ++ return authString == null ? Authorizations.EMPTY : new Authorizations(authString.getBytes(Constants.UTF8)); } - + /** - * Sets the input ranges to scan for this job. If not set, the entire table will be scanned. + * Sets the input ranges to scan on all input tables for this job. If not set, the entire table will be scanned. * * @param implementingClass * the class whose name will be used as a prefix for the property configuration key http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java index 1974f74,c97e2b4..2bc9436 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java @@@ -17,14 -17,16 +17,15 @@@ package org.apache.accumulo.core.client.mock; import java.io.ByteArrayInputStream; -import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.io.Writer; +import java.io.OutputStream; -import jline.ConsoleReader; +import jline.console.ConsoleReader; + import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.util.shell.Shell; -import org.apache.commons.cli.CommandLine; +import org.apache.accumulo.core.util.shell.ShellOptionsJC; /** * An Accumulo Shell implementation that allows a developer to attach an InputStream and Writer to the Shell for testing purposes. @@@ -76,7 -78,7 +77,7 @@@ public class MockShell extends Shell printInfo(); if (execFile != null) { - java.util.Scanner scanner = new java.util.Scanner(execFile); - java.util.Scanner scanner = new java.util.Scanner(new File(execFile), Constants.UTF8.name()); ++ java.util.Scanner scanner = new java.util.Scanner(execFile, Constants.UTF8.name()); try { while (scanner.hasNextLine() && !hasExited()) { execCommand(scanner.nextLine(), true, isVerbose()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java index cfc660e,d653274..6162bc5 --- a/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java +++ b/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java @@@ -20,9 -20,9 +20,10 @@@ import java.io.FileNotFoundException import java.io.IOException; import java.io.InputStream; import java.io.PrintStream; + import java.io.UnsupportedEncodingException; import java.util.ArrayList; -import java.util.Iterator; +import java.util.HashMap; +import java.util.Map; import java.util.Map.Entry; import java.util.TreeMap; http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java index eaa61b9,99e588d..ff56c31 --- a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java +++ b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java @@@ -17,8 -17,9 +17,10 @@@ package org.apache.accumulo.core.data; import java.io.Serializable; +import java.nio.ByteBuffer; + import org.apache.accumulo.core.Constants; + public class ArrayByteSequence extends ByteSequence implements Serializable { private static final long serialVersionUID = 1L; @@@ -46,21 -47,9 +48,21 @@@ } public ArrayByteSequence(String s) { - this(s.getBytes()); + this(s.getBytes(Constants.UTF8)); } + public ArrayByteSequence(ByteBuffer buffer) { + if (buffer.hasArray()) { + this.data = buffer.array(); + this.offset = buffer.arrayOffset(); + this.length = buffer.limit(); + } else { + this.data = new byte[buffer.remaining()]; + this.offset = 0; + buffer.get(data); + } + } + @Override public byte byteAt(int i) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java index eb99807,bd0caba..75091d2 --- a/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java +++ b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java @@@ -124,20 -114,11 +125,21 @@@ public class ColumnVisibility } } + /** + * A node comparator. Nodes sort according to node type, terms sort + * lexicographically. AND and OR nodes sort by number of children, or if + * the same by corresponding children. + */ - public static class NodeComparator implements Comparator { + public static class NodeComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; byte[] text; + /** + * Creates a new comparator. + * + * @param text expression string, encoded in UTF-8 + */ public NodeComparator(byte[] text) { this.text = text; } @@@ -209,9 -190,9 +211,9 @@@ /* * Walks an expression's AST and appends a string representation to a supplied StringBuilder. This method adds parens where necessary. */ - private static void stringify(Node root, byte[] expression, StringBuilder out) { + public static void stringify(Node root, byte[] expression, StringBuilder out) { if (root.type == NodeType.TERM) { - out.append(new String(expression, root.start, root.end - root.start)); + out.append(new String(expression, root.start, root.end - root.start, Constants.UTF8)); } else { String sep = ""; for (Node c : root.children) { @@@ -280,12 -260,12 +282,12 @@@ while (index < expression.length) { switch (expression[index++]) { case '&': { - expr = processTerm(termStart, index - 1, expr, expression); + expr = processTerm(subtermStart, index - 1, expr, expression); if (result != null) { if (!result.type.equals(NodeType.AND)) - throw new BadArgumentException("cannot mix & and |", new String(expression), index - 1); + throw new BadArgumentException("cannot mix & and |", new String(expression, Constants.UTF8), index - 1); } else { - result = new Node(NodeType.AND); + result = new Node(NodeType.AND, wholeTermStart); } result.add(expr); expr = null; @@@ -294,12 -274,12 +296,12 @@@ break; } case '|': { - expr = processTerm(termStart, index - 1, expr, expression); + expr = processTerm(subtermStart, index - 1, expr, expression); if (result != null) { if (!result.type.equals(NodeType.OR)) - throw new BadArgumentException("cannot mix | and &", new String(expression), index - 1); + throw new BadArgumentException("cannot mix | and &", new String(expression, Constants.UTF8), index - 1); } else { - result = new Node(NodeType.OR); + result = new Node(NodeType.OR, wholeTermStart); } result.add(expr); expr = null; @@@ -309,18 -289,18 +311,18 @@@ } case '(': { parens++; - if (termStart != index - 1 || expr != null) + if (subtermStart != index - 1 || expr != null) - throw new BadArgumentException("expression needs & or |", new String(expression), index - 1); + throw new BadArgumentException("expression needs & or |", new String(expression, Constants.UTF8), index - 1); expr = parse_(expression); - termStart = index; - termComplete = false; + subtermStart = index; + subtermComplete = false; break; } case ')': { parens--; - Node child = processTerm(termStart, index - 1, expr, expression); + Node child = processTerm(subtermStart, index - 1, expr, expression); if (child == null && result == null) - throw new BadArgumentException("empty expression not allowed", new String(expression), index); + throw new BadArgumentException("empty expression not allowed", new String(expression, Constants.UTF8), index); if (result == null) return child; if (result.type == child.type) @@@ -332,8 -312,8 +334,8 @@@ return result; } case '"': { - if (termStart != index - 1) + if (subtermStart != index - 1) - throw new BadArgumentException("expression needs & or |", new String(expression), index - 1); + throw new BadArgumentException("expression needs & or |", new String(expression, Constants.UTF8), index - 1); while (index < expression.length && expression[index] != '"') { if (expression[index] == '\\') { @@@ -345,20 -325,20 +347,20 @@@ } if (index == expression.length) - throw new BadArgumentException("unclosed quote", new String(expression), subtermStart); - throw new BadArgumentException("unclosed quote", new String(expression, Constants.UTF8), termStart); - - if (termStart + 1 == index) - throw new BadArgumentException("empty term", new String(expression, Constants.UTF8), termStart); ++ throw new BadArgumentException("unclosed quote", new String(expression, Constants.UTF8), subtermStart); + if (subtermStart + 1 == index) - throw new BadArgumentException("empty term", new String(expression), subtermStart); - ++ throw new BadArgumentException("empty term", new String(expression, Constants.UTF8), subtermStart); ++ index++; - termComplete = true; + subtermComplete = true; break; } default: { - if (termComplete) + if (subtermComplete) - throw new BadArgumentException("expression needs & or |", new String(expression), index - 1); + throw new BadArgumentException("expression needs & or |", new String(expression, Constants.UTF8), index - 1); byte c = expression[index - 1]; if (!Authorizations.isValidAuthChar(c)) http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java index 43d89fa,94aa1a4..8613ca7 --- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java +++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java @@@ -69,10 -61,10 +70,10 @@@ public class VisibilityConstraint imple try { if (ve == null) - ve = new VisibilityEvaluator(env.getAuthorizations()); + ve = new VisibilityEvaluator(env); if (!ve.evaluate(new ColumnVisibility(cv))) - return Collections.singletonList(new Short((short) 2)); + return Collections.singletonList(Short.valueOf((short) 2)); } catch (BadArgumentException bae) { return Collections.singletonList(new Short((short) 1)); http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java index 5b4a9f5,c61faf5..be5f008 --- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java +++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java @@@ -18,23 -18,15 +18,25 @@@ package org.apache.accumulo.core.securi import java.text.ParseException; + import org.apache.accumulo.core.Constants; + +/** + * An exception thrown when a visibility string cannot be parsed. + */ public class VisibilityParseException extends ParseException { private static final long serialVersionUID = 1L; private String visibility; - + + /** + * Creates a new exception. + * + * @param reason reason string + * @param visibility visibility that could not be parsed + * @param errorOffset offset into visibility where parsing failed + */ public VisibilityParseException(String reason, byte[] visibility, int errorOffset) { super(reason, errorOffset); - this.visibility = new String(visibility); + this.visibility = new String(visibility, Constants.UTF8); } @Override http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java index b95aeda,01f76b7..7a23c37 --- a/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/util/ByteBufferUtil.java @@@ -22,7 -22,7 +22,8 @@@ import java.util.Arrays import java.util.Collection; import java.util.List; + import org.apache.accumulo.core.Constants; +import org.apache.accumulo.core.data.ByteSequence; import org.apache.hadoop.io.Text; public class ByteBufferUtil { @@@ -61,18 -61,6 +62,18 @@@ } public static String toString(ByteBuffer bytes) { - return new String(bytes.array(), bytes.position(), bytes.remaining()); + return new String(bytes.array(), bytes.position(), bytes.remaining(), Constants.UTF8); } + + public static ByteBuffer toByteBuffers(ByteSequence bs) { + if (bs == null) + return null; + + if (bs.isBackedByArray()) { + return ByteBuffer.wrap(bs.getBackingArray(), bs.offset(), bs.length()); + } else { + // TODO create more efficient impl + return ByteBuffer.wrap(bs.toArray()); + } + } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/Merge.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/Merge.java index 13f4288,b6cb07f..f268160 --- a/core/src/main/java/org/apache/accumulo/core/util/Merge.java +++ b/core/src/main/java/org/apache/accumulo/core/util/Merge.java @@@ -236,8 -232,8 +237,8 @@@ public class Merge while (iterator.hasNext()) { Entry entry = iterator.next(); Key key = entry.getKey(); - if (key.getColumnFamily().equals(Constants.METADATA_DATAFILE_COLUMN_FAMILY)) { + if (key.getColumnFamily().equals(DataFileColumnFamily.NAME)) { - String[] sizeEntries = new String(entry.getValue().get()).split(","); + String[] sizeEntries = new String(entry.getValue().get(), Constants.UTF8).split(","); if (sizeEntries.length == 2) { tabletSize += Long.parseLong(sizeEntries[0]); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java index 0c4d731,656f2ae..e4a01f9 --- a/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java +++ b/core/src/main/java/org/apache/accumulo/core/util/shell/Shell.java @@@ -492,14 -436,11 +495,14 @@@ public class Shell extends ShellOption } catch (IOException e) { log.warn("Unable to load history file at " + historyPath); } - + + // Turn Ctrl+C into Exception instead of JVM exit + reader.setHandleUserInterrupt(true); + ShellCompletor userCompletor = null; - + if (execFile != null) { - java.util.Scanner scanner = new java.util.Scanner(execFile); - java.util.Scanner scanner = new java.util.Scanner(new File(execFile), Constants.UTF8.name()); ++ java.util.Scanner scanner = new java.util.Scanner(execFile, Constants.UTF8.name()); try { while (scanner.hasNextLine() && !hasExited()) { execCommand(scanner.nextLine(), true, isVerbose()); @@@ -900,14 -803,14 +903,14 @@@ @Override public void close() {} }; - + public static class PrintFile implements PrintLine { PrintWriter writer; - + public PrintFile(String filename) throws FileNotFoundException { - writer = new PrintWriter(filename); + writer = new PrintWriter(new BufferedWriter(new OutputStreamWriter(new FileOutputStream(filename), Constants.UTF8))); } - + @Override public void print(String s) { writer.println(s); @@@ -1037,9 -941,9 +1040,9 @@@ private final void printHelp(String usage, String description, Options opts) { printHelp(usage, description, opts, Integer.MAX_VALUE); } - + private final void printHelp(String usage, String description, Options opts, int width) { - PrintWriter pw = new PrintWriter(System.err); + PrintWriter pw = new PrintWriter(new OutputStreamWriter(System.err, Constants.UTF8)); new HelpFormatter().printHelp(pw, width, usage, description, opts, 2, 5, null, true); pw.flush(); if (logErrorsToConsole && writer != null) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/AuthenticateCommand.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/shell/commands/AuthenticateCommand.java index 362c40a,5266337..bcb6c24 --- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/AuthenticateCommand.java +++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/AuthenticateCommand.java @@@ -34,12 -35,12 +35,12 @@@ public class AuthenticateCommand extend final String user = cl.getArgs()[0]; final String p = shellState.readMaskedLine("Enter current password for '" + user + "': ", '*'); if (p == null) { - shellState.getReader().printNewline(); + shellState.getReader().println(); return 0; } // user canceled - final byte[] password = p.getBytes(); + final byte[] password = p.getBytes(Constants.UTF8); final boolean valid = shellState.getConnector().securityOperations().authenticateUser(user, new PasswordToken(password)); - shellState.getReader().printString((valid ? "V" : "Not v") + "alid\n"); + shellState.getReader().println((valid ? "V" : "Not v") + "alid"); return 0; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java index 1d6640a,b476ae4..25b92be --- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java +++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/CreateTableCommand.java @@@ -54,31 -51,29 +55,30 @@@ public class CreateTableCommand extend private Option createTableOptEVC; private Option base64Opt; private Option createTableOptFormatter; - public static String testTable; - - public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, TableExistsException, - TableNotFoundException, IOException, ClassNotFoundException { - + + @Override + public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws AccumuloException, AccumuloSecurityException, + TableExistsException, TableNotFoundException, IOException, ClassNotFoundException { + final String testTableName = cl.getArgs()[0]; - - if (!testTableName.matches(Constants.VALID_TABLE_NAME_REGEX)) { - shellState.getReader().printString("Only letters, numbers and underscores are allowed for use in table names. \n"); + + if (!testTableName.matches(Tables.VALID_NAME_REGEX)) { + shellState.getReader().println("Only letters, numbers and underscores are allowed for use in table names."); throw new IllegalArgumentException(); } - + final String tableName = cl.getArgs()[0]; if (shellState.getConnector().tableOperations().exists(tableName)) { throw new TableExistsException(null, tableName, null); } final SortedSet partitions = new TreeSet(); final boolean decode = cl.hasOption(base64Opt.getOpt()); - + if (cl.hasOption(createTableOptSplit.getOpt())) { final String f = cl.getOptionValue(createTableOptSplit.getOpt()); - + String line; - Scanner file = new Scanner(new File(f)); + Scanner file = new Scanner(new File(f), Constants.UTF8.name()); try { while (file.hasNextLine()) { line = file.nextLine(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/GetSplitsCommand.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/HiddenCommand.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/shell/commands/HiddenCommand.java index 4cd9bea,41594ca..c212c75 --- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/HiddenCommand.java +++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/HiddenCommand.java @@@ -38,11 -39,12 +39,11 @@@ public class HiddenCommand extends Comm public int execute(final String fullCommand, final CommandLine cl, final Shell shellState) throws Exception { if (rand.nextInt(10) == 0) { shellState.getReader().beep(); - shellState.getReader().printNewline(); - shellState.getReader().printString( + shellState.getReader().println(); + shellState.getReader().println( new String(Base64.decodeBase64(("ICAgICAgIC4tLS4KICAgICAgLyAvXCBcCiAgICAgKCAvLS1cICkKICAgICAuPl8gIF88LgogICAgLyB8ICd8ICcgXAog" + "ICAvICB8Xy58Xy4gIFwKICAvIC98ICAgICAgfFwgXAogfCB8IHwgfFwvfCB8IHwgfAogfF98IHwgfCAgfCB8IHxffAogICAgIC8gIF9fICBcCiAgICAvICAv" - + "ICBcICBcCiAgIC8gIC8gICAgXCAgXF8KIHwvICAvICAgICAgXCB8IHwKIHxfXy8gICAgICAgIFx8X3wK").getBytes()))); + + "ICBcICBcCiAgIC8gIC8gICAgXCAgXF8KIHwvICAvICAgICAgXCB8IHwKIHxfXy8gICAgICAgIFx8X3wK").getBytes(Constants.UTF8)), Constants.UTF8)); - shellState.getReader().printNewline(); } else { throw new ShellCommandException(ErrorCode.UNRECOGNIZED_COMMAND, getName()); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java index 6c93e76,6001956..9aac389 --- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java +++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/ListIterCommand.java @@@ -61,28 -49,22 +61,30 @@@ public class ListIterCommand extends Co iterators.clear(); iterators.put(name, scopes); } - - boolean hasScope = false; + + final boolean allScopes = cl.hasOption(allScopesOpt.getOpt()); + Set desiredScopes = new HashSet(); for (IteratorScope scope : IteratorScope.values()) { - if (cl.hasOption(scopeOpts.get(scope).getOpt())) - hasScope = true; + if (allScopes || cl.hasOption(scopeOpts.get(scope).getOpt())) + desiredScopes.add(scope); } - if (!hasScope) { + if (desiredScopes.isEmpty()) { throw new IllegalArgumentException("You must select at least one scope to configure"); - } + } final StringBuilder sb = new StringBuilder("-\n"); - for (String name : iterators.keySet()) { - for (IteratorScope scope : iterators.get(name)) { + for (Entry> entry : iterators.entrySet()) { + final String name = entry.getKey(); + final EnumSet scopes = entry.getValue(); + for (IteratorScope scope : scopes) { - if (cl.hasOption(scopeOpts.get(scope).getOpt())) { - IteratorSetting setting = shellState.getConnector().tableOperations().getIteratorSetting(tableName, name, scope); + if (desiredScopes.contains(scope)) { + IteratorSetting setting; + if (namespaces) { + setting = shellState.getConnector().namespaceOperations().getIteratorSetting(OptUtil.getNamespaceOpt(cl, shellState), name, scope); + } else if (tables) { + setting = shellState.getConnector().tableOperations().getIteratorSetting(OptUtil.getTableOpt(cl, shellState), name, scope); + } else { + throw new IllegalArgumentException("No table or namespace specified"); + } sb.append("- Iterator ").append(setting.getName()).append(", ").append(scope).append(" scope options:\n"); sb.append("- ").append("iteratorPriority").append(" = ").append(setting.getPriority()).append("\n"); sb.append("- ").append("iteratorClassName").append(" = ").append(setting.getIteratorClass()).append("\n"); http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/PasswdCommand.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java index 2e8d5a4,275b1b0..3d8a5a7 --- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java +++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/SetIterCommand.java @@@ -275,131 -187,85 +275,129 @@@ public class SetIterCommand extends Com } catch (IllegalAccessException e) { throw new IllegalArgumentException(e.getMessage()); } catch (ClassCastException e) { - StringBuilder msg = new StringBuilder("Loaded "); - msg.append(className).append(" but it does not implement "); - msg.append(OptionDescriber.class.getSimpleName()); - msg.append("; use 'config -s' instead."); + StringBuilder msg = new StringBuilder(50); + msg.append(className).append(" loaded successfully but does not implement SortedKeyValueIterator."); + msg.append(" This class cannot be used with this command."); throw new ShellCommandException(ErrorCode.INITIALIZATION_FAILURE, msg.toString()); } - - final IteratorOptions itopts = skvi.describeOptions(); - if (itopts.getName() == null) { - throw new IllegalArgumentException(className + " described its default distinguishing name as null"); - } - String shortClassName = className; - if (className.contains(".")) { - shortClassName = className.substring(className.lastIndexOf('.') + 1); + + @SuppressWarnings("unchecked") + SortedKeyValueIterator skvi = (SortedKeyValueIterator) untypedInstance; + OptionDescriber iterOptions = null; + if (OptionDescriber.class.isAssignableFrom(skvi.getClass())) { + iterOptions = (OptionDescriber) skvi; } - final Map localOptions = new HashMap(); - do { - // clean up the overall options that caused things to fail - for (String key : localOptions.keySet()) { - options.remove(key); - } - localOptions.clear(); - - reader.printString(itopts.getDescription()); - reader.printNewline(); + + String iteratorName; + if (null != iterOptions) { + final IteratorOptions itopts = iterOptions.describeOptions(); + iteratorName = itopts.getName(); - String prompt; - if (itopts.getNamedOptions() != null) { - for (Entry e : itopts.getNamedOptions().entrySet()) { - prompt = Shell.repeat("-", 10) + "> set " + shortClassName + " parameter " + e.getKey() + ", " + e.getValue() + ": "; - reader.flushConsole(); - input = reader.readLine(prompt); - if (input == null) { - reader.printNewline(); - throw new IOException("Input stream closed"); - } - // Places all Parameters and Values into the LocalOptions, even if the value is "". - // This allows us to check for "" values when setting the iterators and allows us to remove - // the parameter and value from the table property. - localOptions.put(e.getKey(), input); - } + if (iteratorName == null) { + throw new IllegalArgumentException(className + " described its default distinguishing name as null"); } - - if (itopts.getUnnamedOptionDescriptions() != null) { - for (String desc : itopts.getUnnamedOptionDescriptions()) { - reader.printString(Shell.repeat("-", 10) + "> entering options: " + desc + "\n"); - input = "start"; - while (true) { - prompt = Shell.repeat("-", 10) + "> set " + shortClassName + " option ( , hit enter to skip): "; - - reader.flushConsole(); + String shortClassName = className; + if (className.contains(".")) { + shortClassName = className.substring(className.lastIndexOf('.') + 1); + } + final Map localOptions = new HashMap(); + do { + // clean up the overall options that caused things to fail + for (String key : localOptions.keySet()) { + options.remove(key); + } + localOptions.clear(); + + reader.println(itopts.getDescription()); + + String prompt; + if (itopts.getNamedOptions() != null) { + for (Entry e : itopts.getNamedOptions().entrySet()) { + prompt = Shell.repeat("-", 10) + "> set " + shortClassName + " parameter " + e.getKey() + ", " + e.getValue() + ": "; + reader.flush(); input = reader.readLine(prompt); if (input == null) { - reader.printNewline(); + reader.println(); throw new IOException("Input stream closed"); - } else { - input = new String(input); } - - if (input.length() == 0) - break; - - String[] sa = input.split(" ", 2); - localOptions.put(sa[0], sa[1]); + // Places all Parameters and Values into the LocalOptions, even if the value is "". + // This allows us to check for "" values when setting the iterators and allows us to remove + // the parameter and value from the table property. + localOptions.put(e.getKey(), input); } } + + if (itopts.getUnnamedOptionDescriptions() != null) { + for (String desc : itopts.getUnnamedOptionDescriptions()) { + reader.println(Shell.repeat("-", 10) + "> entering options: " + desc); + input = "start"; + prompt = Shell.repeat("-", 10) + "> set " + shortClassName + " option ( , hit enter to skip): "; + while (true) { + reader.flush(); + input = reader.readLine(prompt); + if (input == null) { + reader.println(); + throw new IOException("Input stream closed"); + } else { + input = new String(input); + } + + if (input.length() == 0) + break; + + String[] sa = input.split(" ", 2); + localOptions.put(sa[0], sa[1]); + } + } + } + + options.putAll(localOptions); + if (!iterOptions.validateOptions(options)) + reader.println("invalid options for " + clazz.getName()); + + } while (!iterOptions.validateOptions(options)); + } else { + reader.flush(); + reader.println("The iterator class does not implement OptionDescriber. Consider this for better iterator configuration using this setiter command."); + iteratorName = reader.readLine("Name for iterator (enter to skip): "); + if (null == iteratorName) { + reader.println(); + throw new IOException("Input stream closed"); + } else if (StringUtils.isWhitespace(iteratorName)) { + // Treat whitespace or empty string as no name provided + iteratorName = null; } - options.putAll(localOptions); - if (!skvi.validateOptions(options)) - reader.printString("invalid options for " + clazz.getName() + "\n"); + reader.flush(); + reader.println("Optional, configure name-value options for iterator:"); + String prompt = Shell.repeat("-", 10) + "> set option ( , hit enter to skip): "; + final HashMap localOptions = new HashMap(); + + while (true) { + reader.flush(); + input = reader.readLine(prompt); + if (input == null) { + reader.println(); + throw new IOException("Input stream closed"); + } else if (StringUtils.isWhitespace(input)) { + break; + } + + String[] sa = input.split(" ", 2); + localOptions.put(sa[0], sa[1]); + } - } while (!skvi.validateOptions(options)); - return itopts.getName(); + options.putAll(localOptions); + } + + return iteratorName; } - + @Override public String description() { - return "sets a table-specific iterator"; + return "sets a table-specific or namespace-specific iterator"; } - + @Override public Options getOptions() { final Options o = new Options(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java index 8341bff,221beaf..a0743f4 --- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java +++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserCommand.java @@@ -40,10 -41,10 +41,10 @@@ public class UserCommand extends Comman // of these methods fails final String p = shellState.readMaskedLine("Enter password for user " + user + ": ", '*'); if (p == null) { - shellState.getReader().printNewline(); + shellState.getReader().println(); return 0; } // user canceled - pass = p.getBytes(); + pass = p.getBytes(Constants.UTF8); shellState.updateUser(user, new PasswordToken(pass)); return 0; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java ---------------------------------------------------------------------- diff --cc core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java index 48b1853,ce0eed9..a126159 --- a/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java +++ b/core/src/main/java/org/apache/accumulo/core/util/shell/commands/UserPermissionsCommand.java @@@ -45,49 -43,33 +44,52 @@@ public class UserPermissionsCommand ext delim = ", "; } } - shellState.getReader().printNewline(); - + shellState.getReader().println(); + + boolean runOnce = true; + for (String n : shellState.getConnector().namespaceOperations().list()) { + delim = ""; + for (NamespacePermission p : NamespacePermission.values()) { + if (p != null && shellState.getConnector().securityOperations().hasNamespacePermission(user, n, p)) { - if (runOnce == 0) { ++ if (runOnce) { + shellState.getReader().print("\nNamespace permissions (" + n + "): "); - runOnce++; ++ runOnce = false; + } + shellState.getReader().print(delim + "Namespace." + p.name()); + delim = ", "; + } + } - runOnce = 0; ++ runOnce = true; + } + shellState.getReader().println(); + ++ ++ runOnce = true; for (String t : shellState.getConnector().tableOperations().list()) { delim = ""; for (TablePermission p : TablePermission.values()) { if (shellState.getConnector().securityOperations().hasTablePermission(user, t, p) && p != null) { - if (runOnce == 0) { + if (runOnce) { - shellState.getReader().printString("\nTable permissions (" + t + "): "); + shellState.getReader().print("\nTable permissions (" + t + "): "); - runOnce++; + runOnce = false; } - shellState.getReader().printString(delim + "Table." + p.name()); + shellState.getReader().print(delim + "Table." + p.name()); delim = ", "; } - + } - runOnce = 0; + runOnce = true; } - shellState.getReader().printNewline(); + shellState.getReader().println(); + return 0; } - + @Override public String description() { - return "displays a user's system and table permissions"; + return "displays a user's system, table, and namespace permissions"; } - + @Override public Options getOptions() { Options o = new Options(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java ---------------------------------------------------------------------- diff --cc fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java index 8f86d43,0162466..8277c71 --- a/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java +++ b/fate/src/main/java/org/apache/accumulo/fate/AdminUtil.java @@@ -16,10 -16,9 +16,11 @@@ */ package org.apache.accumulo.fate; + import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Collections; +import java.util.EnumSet; +import java.util.Formatter; import java.util.HashMap; import java.util.List; import java.util.Map; @@@ -36,33 -34,9 +37,34 @@@ import org.apache.zookeeper.KeeperExcep * A utility to administer FATE operations */ public class AdminUtil { + private static final Charset UTF8 = Charset.forName("UTF-8"); + private boolean exitOnError = false; + + /** + * Default constructor + */ + public AdminUtil() { + this(true); + } + + /** + * Constructor + * + * @param exitOnError + * System.exit(1) on error if true + */ + public AdminUtil(boolean exitOnError) { + super(); + this.exitOnError = exitOnError; + } + public void print(ZooStore zs, IZooReaderWriter zk, String lockPath) throws KeeperException, InterruptedException { + print(zs, zk, lockPath, new Formatter(System.out), null, null); + } + + public void print(ZooStore zs, IZooReaderWriter zk, String lockPath, Formatter fmt, Set filterTxid, EnumSet filterStatus) + throws KeeperException, InterruptedException { Map> heldLocks = new HashMap>(); Map> waitingLocks = new HashMap>(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/fate/src/main/java/org/apache/accumulo/fate/AgeOffStore.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/fate/src/main/java/org/apache/accumulo/fate/zookeeper/DistributedReadWriteLock.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooCache.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/fate/src/main/java/org/apache/accumulo/fate/zookeeper/ZooSession.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/proxy/src/main/java/org/apache/accumulo/proxy/ProxyServer.java ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/7688eaf0/proxy/src/main/java/org/apache/accumulo/proxy/TestProxyClient.java ----------------------------------------------------------------------