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 7DB5E10579 for ; Tue, 4 Feb 2014 17:55:35 +0000 (UTC) Received: (qmail 26443 invoked by uid 500); 4 Feb 2014 17:54:53 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 25487 invoked by uid 500); 4 Feb 2014 17:54:35 -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 25352 invoked by uid 99); 4 Feb 2014 17:54:30 -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:30 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 4A098824127; Tue, 4 Feb 2014 17:54:30 +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:54:38 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [09/48] git commit: ACCUMULO-2292 Massive amount of fixes for findbugs warnings. ACCUMULO-2292 Massive amount of fixes for findbugs warnings. The strong majority of these changes are all UTF8 related. The rest are split up between using Long.valueOf and Integer.valueOf, Map.entrySet() instead of Map.keySet(), unnecessary String creation/concatenation, and useless conditionals. Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/b11a0361 Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/b11a0361 Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/b11a0361 Branch: refs/heads/1.5.1-SNAPSHOT Commit: b11a03615d583a3f7cc6bebd975d98d3552b83cd Parents: 8f4c500 Author: Josh Elser Authored: Mon Feb 3 21:34:47 2014 -0500 Committer: Josh Elser Committed: Mon Feb 3 21:34:47 2014 -0500 ---------------------------------------------------------------------- .../apache/accumulo/core/cli/ClientOpts.java | 5 +- .../accumulo/core/client/ZooKeeperInstance.java | 8 ++-- .../client/admin/InstanceOperationsImpl.java | 2 +- .../core/client/admin/TableOperationsImpl.java | 39 ++++++++-------- .../accumulo/core/client/impl/ServerClient.java | 12 +++-- .../accumulo/core/client/impl/Tables.java | 6 +-- .../core/client/impl/TabletLocatorImpl.java | 5 +- .../accumulo/core/client/impl/Writer.java | 3 +- .../core/client/mapreduce/InputFormatBase.java | 12 ++--- .../core/client/mapreduce/RangeInputSplit.java | 6 +-- .../lib/partition/RangePartitioner.java | 7 ++- .../mapreduce/lib/util/InputConfigurator.java | 12 ++--- .../accumulo/core/client/mock/MockShell.java | 5 +- .../client/security/tokens/PasswordToken.java | 2 +- .../core/conf/DefaultConfiguration.java | 8 ++-- .../accumulo/core/data/ArrayByteSequence.java | 6 ++- .../org/apache/accumulo/core/data/Column.java | 5 +- .../apache/accumulo/core/data/ColumnUpdate.java | 6 +-- .../org/apache/accumulo/core/data/KeyValue.java | 4 +- .../accumulo/core/file/BloomFilterLayer.java | 5 +- .../core/file/rfile/bcfile/CompareUtils.java | 9 +++- .../core/file/rfile/bcfile/TFileDumper.java | 3 +- .../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 | 3 +- .../iterators/user/RowDeletingIterator.java | 3 +- .../iterators/user/SummingArrayCombiner.java | 5 +- .../iterators/user/TransformingIterator.java | 16 ++++--- .../core/security/ColumnVisibility.java | 38 ++++++++-------- .../core/security/VisibilityConstraint.java | 5 +- .../core/security/VisibilityParseException.java | 4 +- .../core/trace/InstanceUserPassword.java | 3 +- .../apache/accumulo/core/trace/SpanTree.java | 2 +- .../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 | 2 +- .../accumulo/core/util/MetadataTable.java | 6 +-- .../org/apache/accumulo/core/util/TextUtil.java | 2 +- .../apache/accumulo/core/util/shell/Shell.java | 11 +++-- .../util/shell/commands/AddSplitsCommand.java | 5 +- .../shell/commands/AuthenticateCommand.java | 3 +- .../util/shell/commands/CreateTableCommand.java | 5 +- .../util/shell/commands/ExecfileCommand.java | 3 +- .../util/shell/commands/GetSplitsCommand.java | 4 +- .../core/util/shell/commands/HelpCommand.java | 2 +- .../core/util/shell/commands/HiddenCommand.java | 3 +- .../util/shell/commands/HistoryCommand.java | 33 +++++++++----- .../util/shell/commands/ListIterCommand.java | 6 ++- .../core/util/shell/commands/PasswdCommand.java | 3 +- .../shell/commands/QuotedStringTokenizer.java | 3 +- .../util/shell/commands/SetIterCommand.java | 4 -- .../core/util/shell/commands/UserCommand.java | 3 +- .../shell/commands/UserPermissionsCommand.java | 8 ++-- .../org/apache/accumulo/fate/AdminUtil.java | 33 ++++++++++---- .../org/apache/accumulo/fate/AgeOffStore.java | 1 + .../java/org/apache/accumulo/fate/Fate.java | 2 +- .../java/org/apache/accumulo/fate/ZooStore.java | 14 +++--- .../zookeeper/DistributedReadWriteLock.java | 16 ++++--- .../accumulo/fate/zookeeper/ZooCache.java | 6 ++- .../apache/accumulo/fate/zookeeper/ZooLock.java | 9 ++-- .../accumulo/fate/zookeeper/ZooReservation.java | 16 ++++--- .../accumulo/fate/zookeeper/ZooSession.java | 5 +- .../minicluster/MiniAccumuloCluster.java | 16 ++++--- .../org/apache/accumulo/proxy/ProxyServer.java | 17 +++---- .../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 | 10 ++-- .../org/apache/accumulo/server/fate/Admin.java | 8 +++- .../server/gc/SimpleGarbageCollector.java | 6 +-- .../accumulo/server/logger/LogFileValue.java | 12 +++-- .../accumulo/server/logger/LogReader.java | 3 +- .../accumulo/server/master/LiveTServerSet.java | 2 +- .../apache/accumulo/server/master/Master.java | 18 ++++---- .../server/master/recovery/RecoveryManager.java | 2 +- .../server/master/state/DeadServerList.java | 5 +- .../server/master/state/MergeStats.java | 6 +-- .../master/state/MetaDataTableScanner.java | 2 +- .../server/master/state/SetGoalState.java | 2 +- .../server/master/state/TServerInstance.java | 4 +- .../master/state/TabletStateChangeIterator.java | 5 +- .../accumulo/server/master/state/ZooStore.java | 3 +- .../master/state/ZooTabletStateStore.java | 8 ++-- .../master/state/tables/TableManager.java | 20 ++++---- .../server/master/tableOps/BulkImport.java | 6 +-- .../master/tableOps/CancelCompactions.java | 10 ++-- .../server/master/tableOps/CompactRange.java | 16 +++---- .../server/master/tableOps/ExportTable.java | 6 +-- .../server/master/tableOps/ImportTable.java | 10 ++-- .../server/master/tableOps/RenameTable.java | 4 +- .../accumulo/server/master/tableOps/Utils.java | 21 +++++---- .../master/tserverOps/ShutdownTServer.java | 2 +- .../accumulo/server/metanalysis/IndexMeta.java | 6 +-- .../server/metrics/AbstractMetricsImpl.java | 12 +++-- .../accumulo/server/monitor/LogService.java | 2 +- .../apache/accumulo/server/monitor/Monitor.java | 4 +- .../server/monitor/ZooKeeperStatus.java | 5 +- .../server/monitor/servlets/DefaultServlet.java | 3 +- .../server/monitor/servlets/LogServlet.java | 6 +-- .../server/monitor/servlets/ShellServlet.java | 5 +- .../monitor/servlets/TServersServlet.java | 3 +- .../server/monitor/servlets/trace/Basic.java | 3 +- .../server/monitor/servlets/trace/ListType.java | 2 +- .../server/security/SecurityConstants.java | 2 +- .../server/security/SecurityOperation.java | 4 +- .../security/handler/ZKAuthenticator.java | 2 +- .../server/security/handler/ZKAuthorizor.java | 2 +- .../server/security/handler/ZKPermHandler.java | 6 +-- .../tabletserver/BulkFailedCopyProcessor.java | 3 +- .../accumulo/server/tabletserver/Compactor.java | 2 +- .../server/tabletserver/InMemoryMap.java | 5 +- .../accumulo/server/tabletserver/Tablet.java | 17 ++++--- .../server/tabletserver/TabletServer.java | 12 ++--- .../tabletserver/UniqueNameAllocator.java | 8 ++-- .../server/tabletserver/log/DfsLogger.java | 11 +++-- .../metrics/TabletServerScanMetrics.java | 2 +- .../accumulo/server/trace/TraceServer.java | 6 +-- .../server/util/AddFilesWithMissingEntries.java | 2 +- .../accumulo/server/util/ChangeSecret.java | 3 +- .../accumulo/server/util/CleanZookeeper.java | 4 +- .../accumulo/server/util/DeleteZooInstance.java | 4 +- .../accumulo/server/util/DumpZookeeper.java | 2 - .../accumulo/server/util/EmbeddedWebServer.java | 9 ++-- .../org/apache/accumulo/server/util/Info.java | 2 +- .../apache/accumulo/server/util/Initialize.java | 48 ++++++++++---------- .../accumulo/server/util/ListInstances.java | 4 +- .../accumulo/server/util/LocalityCheck.java | 8 ++-- .../accumulo/server/util/MetadataTable.java | 48 ++++++++++---------- .../util/RemoveEntriesForMissingFiles.java | 2 +- .../accumulo/server/util/RestoreZookeeper.java | 5 +- .../accumulo/server/util/SendLogToChainsaw.java | 15 +++--- .../accumulo/server/util/SystemPropUtil.java | 2 +- .../accumulo/server/util/TablePropUtil.java | 2 +- .../accumulo/server/util/TabletServerLocks.java | 2 +- .../server/zookeeper/DistributedWorkQueue.java | 2 +- .../accumulo/server/zookeeper/ZooQueueLock.java | 7 +-- .../server/zookeeper/ZooReaderWriter.java | 3 +- .../start/classloader/AccumuloClassLoader.java | 6 +-- .../vfs/FinalCloseDefaultFileSystemManager.java | 2 +- .../vfs/PostDelegatingVFSClassLoader.java | 2 +- .../vfs/providers/HdfsRandomAccessContent.java | 3 +- .../accumulo/test/BulkImportDirectory.java | 5 +- .../apache/accumulo/test/CreateTestTable.java | 5 +- .../accumulo/test/NativeMapConcurrencyTest.java | 3 +- .../accumulo/test/NativeMapPerformanceTest.java | 7 +-- .../accumulo/test/NativeMapStressTest.java | 5 +- .../apache/accumulo/test/TestBinaryRows.java | 20 ++++---- .../org/apache/accumulo/test/TestIngest.java | 5 +- .../accumulo/test/TestMultiTableIngest.java | 3 +- .../org/apache/accumulo/test/VerifyIngest.java | 3 +- .../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 | 14 +++--- .../accumulo/test/continuous/Histogram.java | 4 +- .../test/continuous/PrintScanTimeHistogram.java | 4 +- .../accumulo/test/continuous/TimeBinner.java | 3 +- .../test/continuous/UndefinedAnalyzer.java | 18 ++++---- .../accumulo/test/functional/AddSplitTest.java | 2 +- .../test/functional/BadIteratorMincTest.java | 4 +- .../test/functional/BatchScanSplitTest.java | 4 +- .../test/functional/BatchWriterFlushTest.java | 2 +- .../test/functional/BloomFilterTest.java | 8 ++-- .../accumulo/test/functional/BulkFileTest.java | 2 +- .../functional/BulkSplitOptimizationTest.java | 3 +- .../test/functional/CacheTestReader.java | 5 +- .../test/functional/CacheTestWriter.java | 9 ++-- .../test/functional/ConcurrencyTest.java | 4 +- .../test/functional/ConstraintTest.java | 20 ++++---- .../test/functional/CreateAndUseTest.java | 2 +- .../test/functional/DeleteEverythingTest.java | 2 +- .../test/functional/DeleteRowsSplitTest.java | 4 +- .../test/functional/FunctionalTest.java | 2 +- .../accumulo/test/functional/LargeRowTest.java | 4 +- .../accumulo/test/functional/NativeMapTest.java | 23 +++++----- .../test/functional/PermissionsTest.java | 6 +-- .../accumulo/test/functional/RunTests.java | 3 +- .../test/functional/ScanIteratorTest.java | 5 +- .../accumulo/test/functional/ScanRangeTest.java | 2 +- .../test/functional/ScanSessionTimeOutTest.java | 3 +- .../test/functional/ServerSideErrorTest.java | 2 +- .../test/functional/SplitRecoveryTest.java | 4 +- .../test/functional/VisibilityTest.java | 9 ++-- .../accumulo/test/functional/ZombieTServer.java | 2 +- .../metadata/MetadataBatchScanTest.java | 4 +- .../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/CheckBalance.java | 2 +- .../randomwalk/concurrent/StopTabletServer.java | 2 +- .../accumulo/test/randomwalk/image/Commit.java | 2 +- .../test/randomwalk/image/ImageFixture.java | 6 +-- .../accumulo/test/randomwalk/image/Verify.java | 7 +-- .../accumulo/test/randomwalk/image/Write.java | 7 +-- .../test/randomwalk/multitable/Commit.java | 2 +- .../test/randomwalk/multitable/CopyTable.java | 2 +- .../test/randomwalk/multitable/CreateTable.java | 2 +- .../multitable/MultiTableFixture.java | 6 +-- .../test/randomwalk/multitable/Write.java | 3 +- .../randomwalk/security/SecurityHelper.java | 7 +-- .../test/randomwalk/security/TableOp.java | 7 +-- .../randomwalk/security/WalkingSecurity.java | 2 +- .../test/randomwalk/sequential/Commit.java | 2 +- .../randomwalk/sequential/MapRedVerifyTool.java | 2 +- .../sequential/SequentialFixture.java | 4 +- .../test/randomwalk/sequential/Write.java | 3 +- .../test/randomwalk/shard/BulkInsert.java | 7 +-- .../test/randomwalk/shard/ExportIndex.java | 3 +- .../accumulo/test/randomwalk/shard/Insert.java | 2 +- .../test/randomwalk/shard/ShardFixture.java | 4 +- .../accumulo/test/scalability/Ingest.java | 3 +- .../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 -- 235 files changed, 843 insertions(+), 656 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java index 7d5d0b8..ff31003 100644 --- a/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java +++ b/core/src/main/java/org/apache/accumulo/core/cli/ClientOpts.java @@ -16,7 +16,6 @@ */ package org.apache.accumulo.core.cli; -import java.nio.charset.Charset; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; @@ -80,12 +79,12 @@ public class ClientOpts extends Help { public byte[] value; public Password(String dfault) { - value = dfault.getBytes(Charset.forName("UTF-8")); + value = dfault.getBytes(Constants.UTF8); } @Override public String toString() { - return new String(value); + return new String(value, Constants.UTF8); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java index 46a7307..5197262 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java +++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java @@ -164,13 +164,13 @@ public class ZooKeeperInstance implements Instance { OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up master location in zoocache."); byte[] loc = ZooUtil.getLockData(zooCache, masterLocPath); - opTimer.stop("Found master at " + (loc == null ? null : new String(loc)) + " in %DURATION%"); + opTimer.stop("Found master at " + (loc == null ? null : new String(loc, Constants.UTF8)) + " in %DURATION%"); if (loc == null) { return Collections.emptyList(); } - return Collections.singletonList(new String(loc)); + return Collections.singletonList(new String(loc, Constants.UTF8)); } @Override @@ -179,13 +179,13 @@ public class ZooKeeperInstance implements Instance { OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Looking up root tablet location in zookeeper."); byte[] loc = zooCache.get(zRootLocPath); - opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc)) + " in %DURATION%"); + opTimer.stop("Found root tablet at " + (loc == null ? null : new String(loc, Constants.UTF8)) + " in %DURATION%"); if (loc == null) { return null; } - return new String(loc).split("\\|")[0]; + return new String(loc, Constants.UTF8).split("\\|")[0]; } @Override http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java index ea833e0..156fa3a 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/admin/InstanceOperationsImpl.java @@ -135,7 +135,7 @@ public class InstanceOperationsImpl implements InstanceOperations { List copy = new ArrayList(children); Collections.sort(copy); byte[] data = cache.get(path + "/" + candidate + "/" + copy.get(0)); - if (data != null && !"master".equals(new String(data))) { + if (data != null && !"master".equals(new String(data, Constants.UTF8))) { results.add(candidate); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/admin/TableOperationsImpl.java ---------------------------------------------------------------------- diff --git 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 index fdb7014..448981b 100644 --- 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 @@ -62,7 +62,6 @@ import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.client.impl.TabletLocator; import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation; import org.apache.accumulo.core.client.impl.thrift.ClientService; -import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode; import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException; import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException; import org.apache.accumulo.core.conf.AccumuloConfiguration; @@ -201,7 +200,7 @@ public class TableOperationsImpl extends TableOperationsHelper { 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); @@ -500,7 +499,7 @@ public class TableOperationsImpl extends TableOperationsHelper { ArgumentChecker.notNull(tableName); ByteBuffer EMPTY = ByteBuffer.allocate(0); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY : TextUtil.getByteBuffer(end)); Map opts = new HashMap(); try { @@ -516,7 +515,7 @@ public class TableOperationsImpl extends TableOperationsHelper { ArgumentChecker.notNull(tableName); ByteBuffer EMPTY = ByteBuffer.allocate(0); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY : TextUtil.getByteBuffer(end)); Map opts = new HashMap(); try { @@ -643,7 +642,7 @@ public class TableOperationsImpl extends TableOperationsHelper { public void delete(String tableName) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { ArgumentChecker.notNull(tableName); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes())); + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8))); Map opts = new HashMap(); try { @@ -675,7 +674,7 @@ public class TableOperationsImpl extends TableOperationsHelper { if (!Collections.disjoint(propertiesToExclude, propertiesToSet.keySet())) throw new IllegalArgumentException("propertiesToSet and propertiesToExclude not disjoint"); - List args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes()), ByteBuffer.wrap(newTableName.getBytes())); + 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)) @@ -710,7 +709,7 @@ public class TableOperationsImpl extends TableOperationsHelper { public void rename(String oldTableName, String newTableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException, TableExistsException { - List args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes()), ByteBuffer.wrap(newTableName.getBytes())); + List args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(newTableName.getBytes(Constants.UTF8))); Map opts = new HashMap(); doTableOperation(TableOperation.RENAME, args, opts); } @@ -764,7 +763,7 @@ public class TableOperationsImpl extends TableOperationsHelper { if (flush) _flush(tableId, start, end, true); - List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes()), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY + List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(Constants.UTF8)), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY : TextUtil.getByteBuffer(end), ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(iterators))); Map opts = new HashMap(); @@ -780,7 +779,7 @@ public class TableOperationsImpl extends TableOperationsHelper { public void cancelCompaction(String tableName) throws AccumuloSecurityException, TableNotFoundException, AccumuloException { 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 { @@ -1110,8 +1109,8 @@ public class TableOperationsImpl extends TableOperationsHelper { throw new AccumuloException("Bulk import failure directory " + failPath + " is not empty"); } - 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 { @@ -1138,7 +1137,7 @@ public class TableOperationsImpl extends TableOperationsHelper { public void offline(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { ArgumentChecker.notNull(tableName); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes())); + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8))); Map opts = new HashMap(); try { @@ -1162,7 +1161,7 @@ public class TableOperationsImpl extends TableOperationsHelper { @Override public void online(String tableName) throws AccumuloSecurityException, AccumuloException, TableNotFoundException { ArgumentChecker.notNull(tableName); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes())); + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8))); Map opts = new HashMap(); try { @@ -1214,7 +1213,7 @@ public class TableOperationsImpl extends TableOperationsHelper { 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); @@ -1236,13 +1235,13 @@ public class TableOperationsImpl extends TableOperationsHelper { try { FileSystem fs = FileUtil.getFileSystem(CachedConfiguration.getInstance(), instance.getConfiguration()); - ; + Map props = getExportedProps(fs, new Path(importDir, Constants.EXPORT_FILE)); - for (String propKey : props.keySet()) { - if (Property.isClassProperty(propKey) && !props.get(propKey).contains(Constants.CORE_PACKAGE_NAME)) { + for (Entry prop : props.entrySet()) { + if (Property.isClassProperty(prop.getKey()) && !prop.getValue().contains(Constants.CORE_PACKAGE_NAME)) { Logger.getLogger(this.getClass()).info( - "Imported table sets '" + propKey + "' to '" + props.get(propKey) + "'. Ensure this class is on Accumulo classpath."); + "Imported table sets '" + prop.getKey() + "' to '" + prop.getValue() + "'. Ensure this class is on Accumulo classpath."); } } @@ -1250,7 +1249,7 @@ public class TableOperationsImpl extends TableOperationsHelper { Logger.getLogger(this.getClass()).warn("Failed to check if imported table references external java classes : " + ioe.getMessage()); } - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(importDir.getBytes())); + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(importDir.getBytes(Constants.UTF8))); Map opts = Collections.emptyMap(); @@ -1267,7 +1266,7 @@ public class TableOperationsImpl extends TableOperationsHelper { public void exportTable(String tableName, String exportDir) throws TableNotFoundException, AccumuloException, AccumuloSecurityException { ArgumentChecker.notNull(tableName, exportDir); - List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes()), ByteBuffer.wrap(exportDir.getBytes())); + List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(Constants.UTF8)), ByteBuffer.wrap(exportDir.getBytes(Constants.UTF8))); Map opts = Collections.emptyMap(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java ---------------------------------------------------------------------- diff --git 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 index 0e301a5..5b97941 100644 --- 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 @@ -140,11 +140,13 @@ 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")) - servers.add(new ThriftTransportKey( - new ServerServices(new String(data)).getAddressString(Service.TSERV_CLIENT), - conf.getPort(Property.TSERV_CLIENTPORT), - rpcTimeout)); + 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)); + } + } } boolean opened = false; http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/impl/Tables.java ---------------------------------------------------------------------- diff --git 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 index d1b10d4..07f75e0 100644 --- 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 @@ -53,9 +53,9 @@ public class Tables { byte[] tblPath = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME); if (tblPath != null) { if (nameAsKey) - tableMap.put(new String(tblPath), tableId); + tableMap.put(new String(tblPath, Constants.UTF8), tableId); else - tableMap.put(tableId, new String(tblPath)); + tableMap.put(tableId, new String(tblPath, Constants.UTF8)); } } @@ -112,7 +112,7 @@ public class Tables { if (state == null) return TableState.UNKNOWN; - return TableState.valueOf(new String(state)); + return TableState.valueOf(new String(state, Constants.UTF8)); } public static long getCacheResetCount() { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java index ea9c73d..f510549 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocatorImpl.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.core.client.impl; +import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -57,8 +58,10 @@ public class TabletLocatorImpl extends TabletLocator { // putting null, put MAX_TEXT static final Text MAX_TEXT = new Text(); - private static class EndRowComparator implements Comparator { + private static class EndRowComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; + public int compare(Text o1, Text o2) { int ret; http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java ---------------------------------------------------------------------- diff --git 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 index cec700d..7353e54 100644 --- 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 @@ -17,6 +17,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; @@ -87,7 +88,7 @@ public class Writer { TabletLocation tabLoc = TabletLocator.getInstance(instance, table).locateTablet(new Text(m.getRow()), false, true, credentials); 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/b11a0361/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java index cc5071c..4516176 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputFormatBase.java @@ -1180,8 +1180,8 @@ public abstract class InputFormatBase extends InputFormat { List iteratorSettings = InputConfigurator.getIterators(CLASS, conf); List deprecatedIterators = new ArrayList(iteratorSettings.size()); for (IteratorSetting setting : iteratorSettings) { - AccumuloIterator deprecatedIter = new AccumuloIterator(new String(setting.getPriority() + AccumuloIterator.FIELD_SEP + setting.getIteratorClass() - + AccumuloIterator.FIELD_SEP + setting.getName())); + AccumuloIterator deprecatedIter = new AccumuloIterator(setting.getPriority() + AccumuloIterator.FIELD_SEP + setting.getIteratorClass() + + AccumuloIterator.FIELD_SEP + setting.getName()); deprecatedIterators.add(deprecatedIter); } return deprecatedIterators; @@ -1198,8 +1198,8 @@ public abstract class InputFormatBase extends InputFormat { for (Entry opt : setting.getOptions().entrySet()) { String deprecatedOption; try { - deprecatedOption = new String(setting.getName() + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getKey(), "UTF-8") - + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getValue(), "UTF-8")); + deprecatedOption = setting.getName() + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getKey(), "UTF-8") + + AccumuloIteratorOption.FIELD_SEP + URLEncoder.encode(opt.getValue(), "UTF-8"); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } @@ -1250,7 +1250,7 @@ public abstract class InputFormatBase extends InputFormat { @Override public String toString() { - return new String(priority + FIELD_SEP + iteratorClass + FIELD_SEP + iteratorName); + return priority + FIELD_SEP + iteratorClass + FIELD_SEP + iteratorName; } } @@ -1299,7 +1299,7 @@ public abstract class InputFormatBase extends InputFormat { @Override public String toString() { try { - return new String(iteratorName + FIELD_SEP + URLEncoder.encode(key, "UTF-8") + FIELD_SEP + URLEncoder.encode(value, "UTF-8")); + return iteratorName + FIELD_SEP + URLEncoder.encode(key, "UTF-8") + FIELD_SEP + URLEncoder.encode(value, "UTF-8"); } catch (UnsupportedEncodingException e) { throw new RuntimeException(e); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/mapreduce/RangeInputSplit.java ---------------------------------------------------------------------- diff --git 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 index 75f140b..73c9b59 100644 --- 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 @@ -67,7 +67,7 @@ public class RangeInputSplit extends InputSplit implements Writable { public RangeInputSplit(Range range, String[] locations) { this.range = range; - this.locations = locations; + setLocations(locations); } public Range getRange() { @@ -138,7 +138,7 @@ public class RangeInputSplit extends InputSplit implements Writable { @Override public String[] getLocations() throws IOException { - return locations; + return Arrays.copyOf(locations, locations.length); } @Override @@ -368,7 +368,7 @@ public class RangeInputSplit extends InputSplit implements Writable { } public void setLocations(String[] locations) { - this.locations = locations; + this.locations = Arrays.copyOf(locations, locations.length); } public Boolean isMockInstance() { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/partition/RangePartitioner.java ---------------------------------------------------------------------- diff --git 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 index 920bd5b..e29c791 100644 --- 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 @@ -17,14 +17,17 @@ package org.apache.accumulo.core.client.mapreduce.lib.partition; import java.io.BufferedReader; +import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.FileReader; import java.io.IOException; +import java.io.InputStreamReader; import java.net.URI; import java.util.Arrays; import java.util.Scanner; import java.util.TreeSet; +import org.apache.accumulo.core.Constants; import org.apache.commons.codec.binary.Base64; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; @@ -87,10 +90,10 @@ public class RangePartitioner extends Partitioner implements Conf for (Path path : cf) { if (path.toUri().getPath().endsWith(cutFileName.substring(cutFileName.lastIndexOf('/')))) { TreeSet cutPoints = new TreeSet(); - Scanner in = new Scanner(new BufferedReader(new FileReader(path.toString()))); + Scanner in = new Scanner(new BufferedReader(new InputStreamReader(new FileInputStream(path.toString()), Constants.UTF8))); try { while (in.hasNextLine()) - cutPoints.add(new Text(Base64.decodeBase64(in.nextLine().getBytes()))); + cutPoints.add(new Text(Base64.decodeBase64(in.nextLine().getBytes(Constants.UTF8)))); } finally { in.close(); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/util/InputConfigurator.java ---------------------------------------------------------------------- diff --git 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 index b0e649b..d9e313c 100644 --- 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 @@ -138,7 +138,7 @@ public class InputConfigurator extends ConfiguratorBase { */ 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 ? Constants.NO_AUTHS : new Authorizations(authString.getBytes(Constants.UTF8)); } /** @@ -216,9 +216,9 @@ public class InputConfigurator extends ConfiguratorBase { if (column.getFirst() == null) throw new IllegalArgumentException("Column family can not be null"); - String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst()))); + String col = new String(Base64.encodeBase64(TextUtil.getBytes(column.getFirst())), Constants.UTF8); if (column.getSecond() != null) - col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond()))); + col += ":" + new String(Base64.encodeBase64(TextUtil.getBytes(column.getSecond())), Constants.UTF8); columnStrings.add(col); } @@ -258,8 +258,8 @@ public class InputConfigurator extends ConfiguratorBase { for (String col : serialized) { int idx = col.indexOf(":"); - Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(Charset.forName("UTF-8"))) : Base64.decodeBase64(col.substring(0, idx).getBytes(Charset.forName("UTF-8")))); - Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes())); + Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(Constants.UTF8)) : Base64.decodeBase64(col.substring(0, idx).getBytes(Constants.UTF8))); + Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes(Constants.UTF8))); columns.add(new Pair(cf, cq)); } return columns; @@ -323,7 +323,7 @@ public class InputConfigurator extends ConfiguratorBase { try { while (tokens.hasMoreTokens()) { String itstring = tokens.nextToken(); - ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(itstring.getBytes())); + ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(itstring.getBytes(Constants.UTF8))); list.add(new IteratorSetting(new DataInputStream(bais))); bais.close(); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/mock/MockShell.java ---------------------------------------------------------------------- diff --git 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 index 4469d5c..c97e2b4 100644 --- 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 @@ -24,6 +24,7 @@ import java.io.Writer; import jline.ConsoleReader; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.util.shell.Shell; import org.apache.commons.cli.CommandLine; @@ -77,7 +78,7 @@ public class MockShell extends Shell { printInfo(); if (execFile != null) { - java.util.Scanner scanner = new java.util.Scanner(new File(execFile)); + java.util.Scanner scanner = new java.util.Scanner(new File(execFile), Constants.UTF8.name()); try { while (scanner.hasNextLine() && !hasExited()) { execCommand(scanner.nextLine(), true, isVerbose()); @@ -137,6 +138,6 @@ public class MockShell extends Shell { sb.append(command).append(NEWLINE); } - return new ByteArrayInputStream(sb.toString().getBytes()); + return new ByteArrayInputStream(sb.toString().getBytes(Constants.UTF8)); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java index 50d6938..9386b56 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java +++ b/core/src/main/java/org/apache/accumulo/core/client/security/tokens/PasswordToken.java @@ -40,7 +40,7 @@ public class PasswordToken implements AuthenticationToken { private byte[] password = null; public byte[] getPassword() { - return password; + return Arrays.copyOf(password, password.length); } /** http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/DefaultConfiguration.java index 58f446d..d653274 100644 --- 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,11 +20,13 @@ 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.Map.Entry; import java.util.TreeMap; +import org.apache.accumulo.core.Constants; import org.apache.log4j.Logger; public class DefaultConfiguration extends AccumuloConfiguration { @@ -62,7 +64,7 @@ public class DefaultConfiguration extends AccumuloConfiguration { int n; try { while ((n = data.read(buffer)) > 0) - doc.print(new String(buffer, 0, n)); + doc.print(new String(buffer, 0, n, Constants.UTF8)); } catch (IOException e) { e.printStackTrace(); return; @@ -176,9 +178,9 @@ public class DefaultConfiguration extends AccumuloConfiguration { /* * Generate documentation for conf/accumulo-site.xml file usage */ - public static void main(String[] args) throws FileNotFoundException { + public static void main(String[] args) throws FileNotFoundException, UnsupportedEncodingException { if (args.length == 2 && args[0].equals("--generate-doc")) { - generateDocumentation(new PrintStream(args[1])); + generateDocumentation(new PrintStream(args[1], Constants.UTF8.name())); } else { throw new IllegalArgumentException("Usage: " + DefaultConfiguration.class.getName() + " --generate-doc "); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java index d44a7a6..99e588d 100644 --- a/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java +++ b/core/src/main/java/org/apache/accumulo/core/data/ArrayByteSequence.java @@ -18,6 +18,8 @@ package org.apache.accumulo.core.data; import java.io.Serializable; +import org.apache.accumulo.core.Constants; + public class ArrayByteSequence extends ByteSequence implements Serializable { private static final long serialVersionUID = 1L; @@ -45,7 +47,7 @@ public class ArrayByteSequence extends ByteSequence implements Serializable { } public ArrayByteSequence(String s) { - this(s.getBytes()); + this(s.getBytes(Constants.UTF8)); } @Override @@ -103,6 +105,6 @@ public class ArrayByteSequence extends ByteSequence implements Serializable { } public String toString() { - return new String(data, offset, length); + return new String(data, offset, length, Constants.UTF8); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/data/Column.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/data/Column.java b/core/src/main/java/org/apache/accumulo/core/data/Column.java index a56c01d..054899f 100644 --- a/core/src/main/java/org/apache/accumulo/core/data/Column.java +++ b/core/src/main/java/org/apache/accumulo/core/data/Column.java @@ -23,6 +23,7 @@ import java.io.DataOutput; import java.io.IOException; import java.nio.ByteBuffer; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.data.thrift.TColumn; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.io.WritableComparator; @@ -158,8 +159,8 @@ public class Column implements WritableComparable { } public String toString() { - return new String(columnFamily == null ? new byte[0] : columnFamily) + ":" + new String(columnQualifier == null ? new byte[0] : columnQualifier) + ":" - + new String(columnVisibility == null ? new byte[0] : columnVisibility); + return new String(columnFamily == null ? new byte[0] : columnFamily, Constants.UTF8) + ":" + new String(columnQualifier == null ? new byte[0] : columnQualifier, Constants.UTF8) + ":" + + new String(columnVisibility == null ? new byte[0] : columnVisibility, Constants.UTF8); } public TColumn toThrift() { http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java b/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java index 691ec0e..bfba00f 100644 --- a/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java +++ b/core/src/main/java/org/apache/accumulo/core/data/ColumnUpdate.java @@ -87,8 +87,8 @@ public class ColumnUpdate { @Override public String toString() { - return new String(Arrays.toString(columnFamily)) + ":" + new String(Arrays.toString(columnQualifier)) + " [" - + new String(Arrays.toString(columnVisibility)) + "] " + (hasTimestamp ? timestamp : "NO_TIME_STAMP") + " " + Arrays.toString(val) + " " + deleted; + return Arrays.toString(columnFamily) + ":" + Arrays.toString(columnQualifier) + " [" + + Arrays.toString(columnVisibility) + "] " + (hasTimestamp ? timestamp : "NO_TIME_STAMP") + " " + Arrays.toString(val) + " " + deleted; } @Override @@ -104,7 +104,7 @@ public class ColumnUpdate { @Override public int hashCode() { return Arrays.hashCode(columnFamily) + Arrays.hashCode(columnQualifier) + Arrays.hashCode(columnVisibility) - + (hasTimestamp ? (Boolean.TRUE.hashCode() + new Long(timestamp).hashCode()) : Boolean.FALSE.hashCode()) + + (hasTimestamp ? (Boolean.TRUE.hashCode() + Long.valueOf(timestamp).hashCode()) : Boolean.FALSE.hashCode()) + (deleted ? Boolean.TRUE.hashCode() : (Boolean.FALSE.hashCode() + Arrays.hashCode(val))); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java b/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java index cc48322..90b2d7c 100644 --- a/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java +++ b/core/src/main/java/org/apache/accumulo/core/data/KeyValue.java @@ -21,6 +21,8 @@ import static org.apache.accumulo.core.util.ByteBufferUtil.toBytes; import java.nio.ByteBuffer; import java.util.Map; +import org.apache.accumulo.core.Constants; + public class KeyValue implements Map.Entry { public Key key; @@ -52,7 +54,7 @@ public class KeyValue implements Map.Entry { } public String toString() { - return key + " " + new String(value); + return key + " " + new String(value, Constants.UTF8); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java index e79da37..01485d3 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java +++ b/core/src/main/java/org/apache/accumulo/core/file/BloomFilterLayer.java @@ -36,6 +36,7 @@ import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.bloomfilter.DynamicBloomFilter; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.ConfigurationCopy; @@ -433,8 +434,8 @@ public class BloomFilterLayer { for (Integer i : vals) { String fi = String.format("%010d", i); - bmfw.append(new org.apache.accumulo.core.data.Key(new Text("r" + fi), new Text("cf1")), new Value(("v" + fi).getBytes())); - bmfw.append(new org.apache.accumulo.core.data.Key(new Text("r" + fi), new Text("cf2")), new Value(("v" + fi).getBytes())); + bmfw.append(new org.apache.accumulo.core.data.Key(new Text("r" + fi), new Text("cf1")), new Value(("v" + fi).getBytes(Constants.UTF8))); + bmfw.append(new org.apache.accumulo.core.data.Key(new Text("r" + fi), new Text("cf2")), new Value(("v" + fi).getBytes(Constants.UTF8))); } long t2 = System.currentTimeMillis(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java index 9dba4b1..0e2bb17 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java +++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/CompareUtils.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.core.file.rfile.bcfile; +import java.io.Serializable; import java.util.Comparator; import org.apache.hadoop.io.RawComparator; @@ -68,7 +69,9 @@ class CompareUtils { } } - public static final class ScalarComparator implements Comparator { + public static final class ScalarComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; + @Override public int compare(Scalar o1, Scalar o2) { long diff = o1.magnitude() - o2.magnitude(); @@ -80,7 +83,9 @@ class CompareUtils { } } - public static final class MemcmpRawComparator implements RawComparator { + public static final class MemcmpRawComparator implements RawComparator, Serializable { + private static final long serialVersionUID = 1L; + @Override public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) { return WritableComparator.compareBytes(b1, s1, l1, b2, s2, l2); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java index d5b0a1b..9b9cd51 100644 --- a/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java +++ b/core/src/main/java/org/apache/accumulo/core/file/rfile/bcfile/TFileDumper.java @@ -24,6 +24,7 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.file.rfile.bcfile.BCFile.BlockRegion; import org.apache.accumulo.core.file.rfile.bcfile.BCFile.MetaIndexEntry; import org.apache.accumulo.core.file.rfile.bcfile.TFile.TFileIndexEntry; @@ -208,7 +209,7 @@ class TFileDumper { out.printf("%X", b); } } else { - out.print(new String(key, 0, sampleLen)); + out.print(new String(key, 0, sampleLen, Constants.UTF8)); } if (sampleLen < key.length) { out.print("..."); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java b/core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java index f780868..b187808 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/ColumnFamilyCounter.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Collection; import java.util.Map; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Range; @@ -58,7 +59,7 @@ public class ColumnFamilyCounter implements SortedKeyValueIterator { } this.key = new Key(currentRow.toArray(), currentColf.toArray(), new byte[0], new byte[0], ts); - this.value = new Value(Integer.toString(count).getBytes()); + this.value = new Value(Integer.toString(count).getBytes(Constants.UTF8)); } else { this.key = null; http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java index 049f854..1d58c2f 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/IteratorUtil.java @@ -17,6 +17,7 @@ package org.apache.accumulo.core.iterators; import java.io.IOException; +import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -54,8 +55,9 @@ public class IteratorUtil { majc, minc, scan; } - public static class IterInfoComparator implements Comparator { - + public static class IterInfoComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; + @Override public int compare(IterInfo o1, IterInfo o2) { return (o1.priority < o2.priority ? -1 : (o1.priority == o2.priority ? 0 : 1)); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java index da23699..a4efec9 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/LongCombiner.java @@ -23,6 +23,7 @@ import java.io.DataOutputStream; import java.io.IOException; import java.util.Map; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; @@ -178,13 +179,13 @@ public abstract class LongCombiner extends TypedValueCombiner { public static class StringEncoder implements Encoder { @Override public byte[] encode(Long v) { - return Long.toString(v).getBytes(); + return Long.toString(v).getBytes(Constants.UTF8); } @Override public Long decode(byte[] b) { try { - return Long.parseLong(new String(b)); + return Long.parseLong(new String(b, Constants.UTF8)); } catch (NumberFormatException nfe) { throw new ValueFormatException(nfe); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java b/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java index 93c1d37..985216c 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/conf/ColumnSet.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.HashSet; import java.util.Set; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.iterators.conf.ColumnUtil.ColFamHashKey; import org.apache.accumulo.core.iterators.conf.ColumnUtil.ColHashKey; @@ -135,7 +136,7 @@ public class ColumnSet { static Text decode(String s) { Text t = new Text(); - byte[] sb = s.getBytes(); + byte[] sb = s.getBytes(Constants.UTF8); // very inefficient code for (int i = 0; i < sb.length; i++) { @@ -143,7 +144,7 @@ public class ColumnSet { t.append(new byte[] {sb[i]}, 0, 1); } else { byte hex[] = new byte[] {sb[++i], sb[++i]}; - String hs = new String(hex); + String hs = new String(hex, Constants.UTF8); int b = Integer.parseInt(hs, 16); t.append(new byte[] {(byte) b}, 0, 1); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java index b037e17..dfea550 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/BigDecimalCombiner.java @@ -21,6 +21,7 @@ import java.math.BigDecimal; import java.util.Iterator; import java.util.Map; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.iterators.IteratorEnvironment; @@ -102,13 +103,13 @@ public abstract class BigDecimalCombiner extends TypedValueCombiner public static class BigDecimalEncoder implements org.apache.accumulo.core.iterators.TypedValueCombiner.Encoder { @Override public byte[] encode(BigDecimal v) { - return v.toString().getBytes(); + return v.toString().getBytes(Constants.UTF8); } @Override public BigDecimal decode(byte[] b) throws ValueFormatException { try { - return new BigDecimal(new String(b)); + return new BigDecimal(new String(b, Constants.UTF8)); } catch (NumberFormatException nfe) { throw new ValueFormatException(nfe); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java index ab8ca84..4f8207c 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/GrepIterator.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Map; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; @@ -88,7 +89,7 @@ public class GrepIterator extends Filter { @Override public void init(SortedKeyValueIterator source, Map options, IteratorEnvironment env) throws IOException { super.init(source, options, env); - term = options.get("term").getBytes(); + term = options.get("term").getBytes(Constants.UTF8); } /** http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java index dbad38f..447200b 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/IntersectingIterator.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Map; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.data.ArrayByteSequence; import org.apache.accumulo.core.data.ByteSequence; @@ -111,7 +112,7 @@ public class IntersectingIterator implements SortedKeyValueIterator { } public String getTermString() { - return (this.term == null) ? new String("Iterator") : this.term.toString(); + return (this.term == null) ? "Iterator" : this.term.toString(); } } @@ -391,7 +392,7 @@ public class IntersectingIterator implements SortedKeyValueIterator { protected static String encodeColumns(Text[] columns) { StringBuilder sb = new StringBuilder(); for (int i = 0; i < columns.length; i++) { - sb.append(new String(Base64.encodeBase64(TextUtil.getBytes(columns[i])))); + sb.append(new String(Base64.encodeBase64(TextUtil.getBytes(columns[i])), Constants.UTF8)); sb.append('\n'); } return sb.toString(); @@ -409,14 +410,14 @@ public class IntersectingIterator implements SortedKeyValueIterator { else bytes[i] = 0; } - return new String(Base64.encodeBase64(bytes)); + return new String(Base64.encodeBase64(bytes), Constants.UTF8); } protected static Text[] decodeColumns(String columns) { String[] columnStrings = columns.split("\n"); Text[] columnTexts = new Text[columnStrings.length]; for (int i = 0; i < columnStrings.length; i++) { - columnTexts[i] = new Text(Base64.decodeBase64(columnStrings[i].getBytes())); + columnTexts[i] = new Text(Base64.decodeBase64(columnStrings[i].getBytes(Constants.UTF8))); } return columnTexts; } @@ -430,7 +431,7 @@ public class IntersectingIterator implements SortedKeyValueIterator { if (flags == null) return null; - byte[] bytes = Base64.decodeBase64(flags.getBytes()); + byte[] bytes = Base64.decodeBase64(flags.getBytes(Constants.UTF8)); boolean[] bFlags = new boolean[bytes.length]; for (int i = 0; i < bytes.length; i++) { if (bytes[i] == 1) http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java index d79604c..fdad848 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/LargeRowFilter.java @@ -23,6 +23,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.Map; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.data.ArrayByteSequence; import org.apache.accumulo.core.data.ByteSequence; @@ -44,7 +45,7 @@ import org.apache.hadoop.io.Text; */ public class LargeRowFilter implements SortedKeyValueIterator, OptionDescriber { - public static final Value SUPPRESS_ROW_VALUE = new Value("SUPPRESS_ROW".getBytes()); + public static final Value SUPPRESS_ROW_VALUE = new Value("SUPPRESS_ROW".getBytes(Constants.UTF8)); private static final ByteSequence EMPTY = new ArrayByteSequence(new byte[] {}); http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java index b110d34..3f5669e 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RegExFilter.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; @@ -192,7 +193,7 @@ public class RegExFilter extends Filter { this.encoding = options.get(ENCODING); if ("".equals(this.encoding)) encoding = ENCODING_DEFAULT; - new String("test".getBytes(), encoding); + new String("test".getBytes(Constants.UTF8), encoding); } catch (UnsupportedEncodingException e) { throw new IllegalArgumentException("invalid encoding " + ENCODING + ":" + this.encoding, e); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java index b139096..45860eb 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/RowDeletingIterator.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.HashSet; import java.util.Map; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.data.ArrayByteSequence; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Key; @@ -48,7 +49,7 @@ import org.apache.accumulo.core.iterators.SortedKeyValueIterator; public class RowDeletingIterator implements SortedKeyValueIterator { - public static final Value DELETE_ROW_VALUE = new Value("DEL_ROW".getBytes()); + public static final Value DELETE_ROW_VALUE = new Value("DEL_ROW".getBytes(Constants.UTF8)); private SortedKeyValueIterator source; private boolean propogateDeletes; private ByteSequence currentRow; http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java index 0ab0cc0..77c829a 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/SummingArrayCombiner.java @@ -27,6 +27,7 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; @@ -207,12 +208,12 @@ public class SummingArrayCombiner extends TypedValueCombiner> { sb.append(","); sb.append(Long.toString(la.get(i))); } - return sb.toString().getBytes(); + return sb.toString().getBytes(Constants.UTF8); } @Override public List decode(byte[] b) { - String[] longstrs = new String(b).split(","); + String[] longstrs = new String(b, Constants.UTF8).split(","); List la = new ArrayList(longstrs.length); for (String s : longstrs) { if (s.length() == 0) http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java b/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java index f439437..8835b1c 100644 --- a/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/iterators/user/TransformingIterator.java @@ -24,8 +24,10 @@ import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.Map.Entry; import java.util.NoSuchElementException; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.data.ByteSequence; @@ -110,7 +112,7 @@ abstract public class TransformingIterator extends WrappingIterator implements O if (scanning) { String auths = options.get(AUTH_OPT); if (auths != null && !auths.isEmpty()) { - ve = new VisibilityEvaluator(new Authorizations(auths.getBytes())); + ve = new VisibilityEvaluator(new Authorizations(auths.getBytes(Constants.UTF8))); visibleCache = new LRUMap(100); } } @@ -141,15 +143,15 @@ abstract public class TransformingIterator extends WrappingIterator implements O @Override public boolean validateOptions(Map options) { - for (String opt : options.keySet()) { + for (Entry option : options.entrySet()) { try { - if (opt.equals(AUTH_OPT)) { - new Authorizations(options.get(opt).getBytes()); - } else if (opt.equals(MAX_BUFFER_SIZE_OPT)) { - AccumuloConfiguration.getMemoryInBytes(options.get(opt)); + if (option.getKey().equals(AUTH_OPT)) { + new Authorizations(option.getValue().getBytes(Constants.UTF8)); + } else if (option.getKey().equals(MAX_BUFFER_SIZE_OPT)) { + AccumuloConfiguration.getMemoryInBytes(option.getValue()); } } catch (Exception e) { - throw new IllegalArgumentException("Failed to parse opt " + opt + " " + options.get(opt), e); + throw new IllegalArgumentException("Failed to parse opt " + option.getKey() + " " + option.getValue(), e); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java index 7d7daa2..bd0caba 100644 --- a/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java +++ b/core/src/main/java/org/apache/accumulo/core/security/ColumnVisibility.java @@ -16,6 +16,7 @@ */ package org.apache.accumulo.core.security; +import java.io.Serializable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -113,8 +114,9 @@ public class ColumnVisibility { } } - public static class NodeComparator implements Comparator { + public static class NodeComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; byte[] text; public NodeComparator(byte[] text) { @@ -190,7 +192,7 @@ public class ColumnVisibility { */ private 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) { @@ -215,7 +217,7 @@ public class ColumnVisibility { Node normRoot = normalize(node, expression); StringBuilder builder = new StringBuilder(expression.length); stringify(normRoot, expression, builder); - return builder.toString().getBytes(); + return builder.toString().getBytes(Constants.UTF8); } private static class ColumnVisibilityParser { @@ -228,10 +230,10 @@ public class ColumnVisibility { if (expression.length > 0) { Node node = parse_(expression); if (node == null) { - throw new BadArgumentException("operator or missing parens", new String(expression), index - 1); + throw new BadArgumentException("operator or missing parens", new String(expression, Constants.UTF8), index - 1); } if (parens != 0) { - throw new BadArgumentException("parenthesis mis-match", new String(expression), index - 1); + throw new BadArgumentException("parenthesis mis-match", new String(expression, Constants.UTF8), index - 1); } return node; } @@ -241,11 +243,11 @@ public class ColumnVisibility { Node processTerm(int start, int end, Node expr, byte[] expression) { if (start != end) { if (expr != null) - throw new BadArgumentException("expression needs | or &", new String(expression), start); + throw new BadArgumentException("expression needs | or &", new String(expression, Constants.UTF8), start); return new Node(start, end); } if (expr == null) - throw new BadArgumentException("empty term", new String(expression), start); + throw new BadArgumentException("empty term", new String(expression, Constants.UTF8), start); return expr; } @@ -261,7 +263,7 @@ public class ColumnVisibility { expr = processTerm(termStart, 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); } @@ -275,7 +277,7 @@ public class ColumnVisibility { expr = processTerm(termStart, 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); } @@ -288,7 +290,7 @@ public class ColumnVisibility { case '(': { parens++; if (termStart != 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; @@ -298,7 +300,7 @@ public class ColumnVisibility { parens--; Node child = processTerm(termStart, 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) @@ -311,22 +313,22 @@ public class ColumnVisibility { } case '"': { if (termStart != 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] == '\\') { index++; if (expression[index] != '\\' && expression[index] != '"') - throw new BadArgumentException("invalid escaping within quotes", new String(expression), index - 1); + throw new BadArgumentException("invalid escaping within quotes", new String(expression, Constants.UTF8), index - 1); } index++; } if (index == expression.length) - throw new BadArgumentException("unclosed quote", new String(expression), termStart); + throw new BadArgumentException("unclosed quote", new String(expression, Constants.UTF8), termStart); if (termStart + 1 == index) - throw new BadArgumentException("empty term", new String(expression), termStart); + throw new BadArgumentException("empty term", new String(expression, Constants.UTF8), termStart); index++; @@ -336,11 +338,11 @@ public class ColumnVisibility { } default: { if (termComplete) - 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)) - throw new BadArgumentException("bad character (" + c + ")", new String(expression), index - 1); + throw new BadArgumentException("bad character (" + c + ")", new String(expression, Constants.UTF8), index - 1); } } } @@ -351,7 +353,7 @@ public class ColumnVisibility { result = child; if (result.type != NodeType.TERM) if (result.children.size() < 2) - throw new BadArgumentException("missing term", new String(expression), index); + throw new BadArgumentException("missing term", new String(expression, Constants.UTF8), index); return result; } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/b11a0361/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java index c8b33ba..94aa1a4 100644 --- a/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java +++ b/core/src/main/java/org/apache/accumulo/core/security/VisibilityConstraint.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.constraints.Constraint; import org.apache.accumulo.core.data.ColumnUpdate; import org.apache.accumulo.core.data.Mutation; @@ -54,7 +55,7 @@ public class VisibilityConstraint implements Constraint { byte[] cv = update.getColumnVisibility(); if (cv.length > 0) { String key = null; - if (ok != null && ok.contains(key = new String(cv))) + if (ok != null && ok.contains(key = new String(cv, Constants.UTF8))) continue; try { @@ -63,7 +64,7 @@ public class VisibilityConstraint implements Constraint { ve = new VisibilityEvaluator(env.getAuthorizations()); 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/b11a0361/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java b/core/src/main/java/org/apache/accumulo/core/security/VisibilityParseException.java index 2f46dc9..c61faf5 100644 --- 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,13 +18,15 @@ package org.apache.accumulo.core.security; import java.text.ParseException; +import org.apache.accumulo.core.Constants; + public class VisibilityParseException extends ParseException { private static final long serialVersionUID = 1L; private String visibility; public VisibilityParseException(String reason, byte[] visibility, int errorOffset) { super(reason, errorOffset); - this.visibility = new String(visibility); + this.visibility = new String(visibility, Constants.UTF8); } @Override