Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id E5D66200BDD for ; Fri, 4 Nov 2016 23:22:50 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id E4729160AFE; Fri, 4 Nov 2016 22:22:50 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id DDE82160B1F for ; Fri, 4 Nov 2016 23:22:47 +0100 (CET) Received: (qmail 68952 invoked by uid 500); 4 Nov 2016 22:22:47 -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 66486 invoked by uid 99); 4 Nov 2016 22:22:44 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 04 Nov 2016 22:22:44 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id EB7FDF171D; Fri, 4 Nov 2016 22:22:43 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ctubbsii@apache.org To: commits@accumulo.apache.org Date: Fri, 04 Nov 2016 22:23:11 -0000 Message-Id: <1c200a0820034dbd858f2ef24aff3a19@git.apache.org> In-Reply-To: <8db15481562846a5947e4d4b91c82ee1@git.apache.org> References: <8db15481562846a5947e4d4b91c82ee1@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [29/48] accumulo git commit: ACCUMULO-4514 Remove unnecessary code archived-at: Fri, 04 Nov 2016 22:22:51 -0000 http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java index 6f9ea29..1e7b4ec 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java @@ -133,7 +133,7 @@ public class TableOperationsImpl extends TableOperationsHelper { @Override public SortedSet list() { OpTimer opTimer = new OpTimer(log, Level.TRACE).start("Fetching list of tables..."); - TreeSet tableNames = new TreeSet(Tables.getNameToIdMap(context.getInstance()).keySet()); + TreeSet tableNames = new TreeSet<>(Tables.getNameToIdMap(context.getInstance()).keySet()); opTimer.stop("Fetched " + tableNames.size() + " table names in %DURATION%"); return tableNames; } @@ -346,7 +346,7 @@ public class TableOperationsImpl extends TableOperationsHelper { return; if (splits.size() <= 2) { - addSplits(env.tableName, new TreeSet(splits), env.tableId); + addSplits(env.tableName, new TreeSet<>(splits), env.tableId); for (int i = 0; i < splits.size(); i++) env.latch.countDown(); return; @@ -356,7 +356,7 @@ public class TableOperationsImpl extends TableOperationsHelper { // split the middle split point to ensure that child task split different tablets and can therefore // run in parallel - addSplits(env.tableName, new TreeSet(splits.subList(mid, mid + 1)), env.tableId); + addSplits(env.tableName, new TreeSet<>(splits.subList(mid, mid + 1)), env.tableId); env.latch.countDown(); env.executor.submit(new SplitTask(env, splits.subList(0, mid))); @@ -373,13 +373,13 @@ public class TableOperationsImpl extends TableOperationsHelper { public void addSplits(String tableName, SortedSet partitionKeys) throws TableNotFoundException, AccumuloException, AccumuloSecurityException { String tableId = Tables.getTableId(context.getInstance(), tableName); - List splits = new ArrayList(partitionKeys); + List splits = new ArrayList<>(partitionKeys); // should be sorted because we copied from a sorted set, but that makes assumptions about // how the copy was done so resort to be sure. Collections.sort(splits); CountDownLatch latch = new CountDownLatch(splits.size()); - AtomicReference exception = new AtomicReference(null); + AtomicReference exception = new AtomicReference<>(null); ExecutorService executor = Executors.newFixedThreadPool(16, new NamingThreadFactory("addSplits")); try { @@ -491,7 +491,7 @@ public class TableOperationsImpl extends TableOperationsHelper { ByteBuffer EMPTY = ByteBuffer.allocate(0); List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY : TextUtil.getByteBuffer(end)); - Map opts = new HashMap(); + Map opts = new HashMap<>(); try { doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_MERGE, args, opts); } catch (TableExistsException e) { @@ -507,7 +507,7 @@ public class TableOperationsImpl extends TableOperationsHelper { ByteBuffer EMPTY = ByteBuffer.allocate(0); List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)), start == null ? EMPTY : TextUtil.getByteBuffer(start), end == null ? EMPTY : TextUtil.getByteBuffer(end)); - Map opts = new HashMap(); + Map opts = new HashMap<>(); try { doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE_RANGE, args, opts); } catch (TableExistsException e) { @@ -523,7 +523,7 @@ public class TableOperationsImpl extends TableOperationsHelper { String tableId = Tables.getTableId(context.getInstance(), tableName); - TreeMap tabletLocations = new TreeMap(); + TreeMap tabletLocations = new TreeMap<>(); while (true) { try { @@ -547,7 +547,7 @@ public class TableOperationsImpl extends TableOperationsHelper { } } - ArrayList endRows = new ArrayList(tabletLocations.size()); + ArrayList endRows = new ArrayList<>(tabletLocations.size()); for (KeyExtent ke : tabletLocations.keySet()) if (ke.getEndRow() != null) @@ -576,7 +576,7 @@ public class TableOperationsImpl extends TableOperationsHelper { double r = (maxSplits + 1) / (double) (endRows.size()); double pos = 0; - ArrayList subset = new ArrayList(maxSplits); + ArrayList subset = new ArrayList<>(maxSplits); int j = 0; for (int i = 0; i < endRows.size() && j < maxSplits; i++) { @@ -606,7 +606,7 @@ public class TableOperationsImpl extends TableOperationsHelper { checkArgument(tableName != null, "tableName is null"); List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8))); - Map opts = new HashMap(); + Map opts = new HashMap<>(); try { doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_DELETE, args, opts); @@ -636,7 +636,7 @@ public class TableOperationsImpl extends TableOperationsHelper { propertiesToSet = Collections.emptyMap(); List args = Arrays.asList(ByteBuffer.wrap(srcTableId.getBytes(UTF_8)), ByteBuffer.wrap(newTableName.getBytes(UTF_8))); - Map opts = new HashMap(); + Map opts = new HashMap<>(); for (Entry entry : propertiesToSet.entrySet()) { if (entry.getKey().startsWith(CLONE_EXCLUDE_PREFIX)) throw new IllegalArgumentException("Property can not start with " + CLONE_EXCLUDE_PREFIX); @@ -655,7 +655,7 @@ public class TableOperationsImpl extends TableOperationsHelper { TableExistsException { List args = Arrays.asList(ByteBuffer.wrap(oldTableName.getBytes(UTF_8)), ByteBuffer.wrap(newTableName.getBytes(UTF_8))); - Map opts = new HashMap(); + Map opts = new HashMap<>(); doTableFateOperation(oldTableName, TableNotFoundException.class, FateOperation.TABLE_RENAME, args, opts); } @@ -723,7 +723,7 @@ public class TableOperationsImpl extends TableOperationsHelper { : TextUtil.getByteBuffer(end), ByteBuffer.wrap(IteratorUtil.encodeIteratorSettings(config.getIterators())), ByteBuffer .wrap(CompactionStrategyConfigUtil.encode(config.getCompactionStrategy()))); - Map opts = new HashMap(); + Map opts = new HashMap<>(); try { doFateOperation(FateOperation.TABLE_COMPACT, args, opts, tableName, config.getWait()); } catch (TableExistsException e) { @@ -743,7 +743,7 @@ public class TableOperationsImpl extends TableOperationsHelper { List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(UTF_8))); - Map opts = new HashMap(); + Map opts = new HashMap<>(); try { doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_CANCEL_COMPACT, args, opts); } catch (TableExistsException e) { @@ -872,7 +872,7 @@ public class TableOperationsImpl extends TableOperationsHelper { @Override public void setLocalityGroups(String tableName, Map> groups) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { // ensure locality groups do not overlap - HashSet all = new HashSet(); + HashSet all = new HashSet<>(); for (Entry> entry : groups.entrySet()) { if (!Collections.disjoint(all, entry.getValue())) { @@ -918,10 +918,10 @@ public class TableOperationsImpl extends TableOperationsHelper { AccumuloConfiguration conf = new ConfigurationCopy(this.getProperties(tableName)); Map> groups = LocalityGroupUtil.getLocalityGroups(conf); - Map> groups2 = new HashMap>(); + Map> groups2 = new HashMap<>(); for (Entry> entry : groups.entrySet()) { - HashSet colFams = new HashSet(); + HashSet colFams = new HashSet<>(); for (ByteSequence bs : entry.getValue()) { colFams.add(new Text(bs.toArray())); @@ -944,7 +944,7 @@ public class TableOperationsImpl extends TableOperationsHelper { return Collections.singleton(range); Random random = new Random(); - Map>> binnedRanges = new HashMap>>(); + Map>> binnedRanges = new HashMap<>(); String tableId = Tables.getTableId(context.getInstance(), tableName); TabletLocator tl = TabletLocator.getLocator(context, new Text(tableId)); // its possible that the cache could contain complete, but old information about a tables tablets... so clear it @@ -963,8 +963,8 @@ public class TableOperationsImpl extends TableOperationsHelper { } // group key extents to get <= maxSplits - LinkedList unmergedExtents = new LinkedList(); - List mergedExtents = new ArrayList(); + LinkedList unmergedExtents = new LinkedList<>(); + List mergedExtents = new ArrayList<>(); for (Map> map : binnedRanges.values()) unmergedExtents.addAll(map.keySet()); @@ -989,7 +989,7 @@ public class TableOperationsImpl extends TableOperationsHelper { mergedExtents.addAll(unmergedExtents); - Set ranges = new HashSet(); + Set ranges = new HashSet<>(); for (KeyExtent k : mergedExtents) ranges.add(k.toDataRange().clip(range)); @@ -1040,7 +1040,7 @@ public class TableOperationsImpl extends TableOperationsHelper { List args = Arrays.asList(ByteBuffer.wrap(tableName.getBytes(UTF_8)), ByteBuffer.wrap(dirPath.toString().getBytes(UTF_8)), ByteBuffer.wrap(failPath.toString().getBytes(UTF_8)), ByteBuffer.wrap((setTime + "").getBytes(UTF_8))); - Map opts = new HashMap(); + Map opts = new HashMap<>(); try { doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_BULK_IMPORT, args, opts); @@ -1087,7 +1087,7 @@ public class TableOperationsImpl extends TableOperationsHelper { int waitFor = 0; int holes = 0; Text continueRow = null; - MapCounter serverCounts = new MapCounter(); + MapCounter serverCounts = new MapCounter<>(); while (rowIter.hasNext()) { Iterator> row = rowIter.next(); @@ -1187,7 +1187,7 @@ public class TableOperationsImpl extends TableOperationsHelper { checkArgument(tableName != null, "tableName is null"); String tableId = Tables.getTableId(context.getInstance(), tableName); List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(UTF_8))); - Map opts = new HashMap(); + Map opts = new HashMap<>(); try { doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_OFFLINE, args, opts); @@ -1210,7 +1210,7 @@ public class TableOperationsImpl extends TableOperationsHelper { checkArgument(tableName != null, "tableName is null"); String tableId = Tables.getTableId(context.getInstance(), tableName); List args = Arrays.asList(ByteBuffer.wrap(tableId.getBytes(UTF_8))); - Map opts = new HashMap(); + Map opts = new HashMap<>(); try { doTableFateOperation(tableName, TableNotFoundException.class, FateOperation.TABLE_ONLINE, args, opts); @@ -1284,16 +1284,16 @@ public class TableOperationsImpl extends TableOperationsHelper { } } - List finalUsages = new ArrayList(); + List finalUsages = new ArrayList<>(); for (TDiskUsage diskUsage : diskUsages) { - finalUsages.add(new DiskUsage(new TreeSet(diskUsage.getTables()), diskUsage.getUsage())); + finalUsages.add(new DiskUsage(new TreeSet<>(diskUsage.getTables()), diskUsage.getUsage())); } return finalUsages; } public static Map getExportedProps(FileSystem fs, Path path) throws IOException { - HashMap props = new HashMap(); + HashMap props = new HashMap<>(); ZipInputStream zis = new ZipInputStream(fs.open(path)); try { http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/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 beacea9..58ebb72 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 @@ -60,8 +60,8 @@ public class Tables { ZooCache zc = getZooCache(instance); List tableIds = zc.getChildren(ZooUtil.getRoot(instance) + Constants.ZTABLES); - TreeMap tableMap = new TreeMap(); - Map namespaceIdToNameMap = new HashMap(); + TreeMap tableMap = new TreeMap<>(); + Map namespaceIdToNameMap = new HashMap<>(); for (String tableId : tableIds) { byte[] tableName = zc.get(ZooUtil.getRoot(instance) + Constants.ZTABLES + "/" + tableId + Constants.ZTABLE_NAME); @@ -211,9 +211,9 @@ public class Tables { tableName = MetadataTable.NAME; if (tableName.contains(".")) { String[] s = tableName.split("\\.", 2); - return new Pair(s[0], s[1]); + return new Pair<>(s[0], s[1]); } - return new Pair(defaultNamespace, tableName); + return new Pair<>(defaultNamespace, tableName); } /** http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java index 1fbaee8..5ecc319 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletLocator.java @@ -90,7 +90,7 @@ public abstract class TabletLocator { } - private static HashMap locators = new HashMap(); + private static HashMap locators = new HashMap<>(); public static synchronized void clearLocators() { locators.clear(); @@ -136,7 +136,7 @@ public abstract class TabletLocator { } public static class TabletLocation implements Comparable { - private static final WeakHashMap> tabletLocs = new WeakHashMap>(); + private static final WeakHashMap> tabletLocs = new WeakHashMap<>(); private static String dedupeLocation(String tabletLoc) { synchronized (tabletLocs) { @@ -148,7 +148,7 @@ public abstract class TabletLocator { } } - tabletLocs.put(tabletLoc, new WeakReference(tabletLoc)); + tabletLocs.put(tabletLoc, new WeakReference<>(tabletLoc)); return tabletLoc; } } @@ -203,13 +203,13 @@ public abstract class TabletLocator { public TabletServerMutations(String tserverSession) { this.tserverSession = tserverSession; - this.mutations = new HashMap>(); + this.mutations = new HashMap<>(); } public void addMutation(KeyExtent ke, T m) { List mutList = mutations.get(ke); if (mutList == null) { - mutList = new ArrayList(); + mutList = new ArrayList<>(); mutations.put(ke, mutList); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/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 c28320d..63f3eb5 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 @@ -88,12 +88,12 @@ public class TabletLocatorImpl extends TabletLocator { protected Text tableId; protected TabletLocator parent; - protected TreeMap metaCache = new TreeMap(endRowComparator); + protected TreeMap metaCache = new TreeMap<>(endRowComparator); protected TabletLocationObtainer locationObtainer; private TabletServerLockChecker lockChecker; protected Text lastTabletRow; - private TreeSet badExtents = new TreeSet(); + private TreeSet badExtents = new TreeSet<>(); private ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); private final Lock rLock = rwLock.readLock(); private final Lock wLock = rwLock.writeLock(); @@ -117,8 +117,8 @@ public class TabletLocatorImpl extends TabletLocator { private class LockCheckerSession { - private HashSet> okLocks = new HashSet>(); - private HashSet> invalidLocks = new HashSet>(); + private HashSet> okLocks = new HashSet<>(); + private HashSet> invalidLocks = new HashSet<>(); private TabletLocation checkLock(TabletLocation tl) { // the goal of this class is to minimize calls out to lockChecker under that assumption that its a resource synchronized among many threads... want to @@ -128,7 +128,7 @@ public class TabletLocatorImpl extends TabletLocator { if (tl == null) return null; - Pair lock = new Pair(tl.tablet_location, tl.tablet_session); + Pair lock = new Pair<>(tl.tablet_location, tl.tablet_session); if (okLocks.contains(lock)) return tl; @@ -168,7 +168,7 @@ public class TabletLocatorImpl extends TabletLocator { if (log.isTraceEnabled()) opTimer = new OpTimer(log, Level.TRACE).start("Binning " + mutations.size() + " mutations for table " + tableId); - ArrayList notInCache = new ArrayList(); + ArrayList notInCache = new ArrayList<>(); Text row = new Text(); LockCheckerSession lcSession = new LockCheckerSession(); @@ -238,7 +238,7 @@ public class TabletLocatorImpl extends TabletLocator { // do lock check once per tserver here to make binning faster boolean lockHeld = lcSession.checkLock(tl) != null; if (lockHeld) { - tsm = new TabletServerMutations(tl.tablet_session); + tsm = new TabletServerMutations<>(tl.tablet_session); binnedMutations.put(tl.tablet_location, tsm); } else { return false; @@ -256,8 +256,8 @@ public class TabletLocatorImpl extends TabletLocator { private List binRanges(ClientContext context, List ranges, Map>> binnedRanges, boolean useCache, LockCheckerSession lcSession) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { - List failures = new ArrayList(); - List tabletLocations = new ArrayList(); + List failures = new ArrayList<>(); + List tabletLocations = new ArrayList<>(); boolean lookupFailed = false; @@ -656,7 +656,7 @@ public class TabletLocatorImpl extends TabletLocator { return; } - List lookups = new ArrayList(badExtents.size()); + List lookups = new ArrayList<>(badExtents.size()); for (KeyExtent be : badExtents) { lookups.add(be.toMetadataRange()); @@ -665,12 +665,12 @@ public class TabletLocatorImpl extends TabletLocator { lookups = Range.mergeOverlapping(lookups); - Map>> binnedRanges = new HashMap>>(); + Map>> binnedRanges = new HashMap<>(); parent.binRanges(context, lookups, binnedRanges); // randomize server order - ArrayList tabletServers = new ArrayList(binnedRanges.keySet()); + ArrayList tabletServers = new ArrayList<>(binnedRanges.keySet()); Collections.shuffle(tabletServers); for (String tserver : tabletServers) { @@ -691,13 +691,13 @@ public class TabletLocatorImpl extends TabletLocator { protected static void addRange(Map>> binnedRanges, String location, KeyExtent ke, Range range) { Map> tablets = binnedRanges.get(location); if (tablets == null) { - tablets = new HashMap>(); + tablets = new HashMap<>(); binnedRanges.put(location, tablets); } List tabletsRanges = tablets.get(ke); if (tabletsRanges == null) { - tabletsRanges = new ArrayList(); + tabletsRanges = new ArrayList<>(); tablets.put(ke, tabletsRanges); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java index 6d09936..5eabf15 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReader.java @@ -98,7 +98,7 @@ public class TabletServerBatchReader extends ScannerOptions implements BatchScan throw new IllegalStateException("batch reader closed"); } - this.ranges = new ArrayList(ranges); + this.ranges = new ArrayList<>(ranges); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java index 053f2b3..3c29c16 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java @@ -87,7 +87,7 @@ public class TabletServerBatchReaderIterator implements Iterator>> resultsQueue; private Iterator> batchIterator; private List> batch; - private static final List> LAST_BATCH = new ArrayList>(); + private static final List> LAST_BATCH = new ArrayList<>(); private final Object nextLock = new Object(); private long failSleepTime = 100; @@ -114,7 +114,7 @@ public class TabletServerBatchReaderIterator implements Iterator>>(numThreads); + resultsQueue = new ArrayBlockingQueue<>(numThreads); this.locator = new TimeoutTabletLocator(TabletLocator.getLocator(context, new Text(table)), timeout); @@ -123,7 +123,7 @@ public class TabletServerBatchReaderIterator implements Iterator 0) { - ArrayList ranges2 = new ArrayList(ranges.size()); + ArrayList ranges2 = new ArrayList<>(ranges.size()); for (Range range : ranges) { ranges2.add(range.bound(options.fetchedColumns.first(), options.fetchedColumns.last())); } @@ -212,10 +212,10 @@ public class TabletServerBatchReaderIterator implements Iterator ranges, ResultReceiver receiver) throws AccumuloException, AccumuloSecurityException, TableNotFoundException { - List columns = new ArrayList(options.fetchedColumns); + List columns = new ArrayList<>(options.fetchedColumns); ranges = Range.mergeOverlapping(ranges); - Map>> binnedRanges = new HashMap>>(); + Map>> binnedRanges = new HashMap<>(); binRanges(locator, ranges, binnedRanges); @@ -260,13 +260,13 @@ public class TabletServerBatchReaderIterator implements Iterator>> binnedRanges2 = new HashMap>>(); + Map>> binnedRanges2 = new HashMap<>(); for (Entry>> entry : binnedRanges.entrySet()) { - Map> tabletMap = new HashMap>(); + Map> tabletMap = new HashMap<>(); binnedRanges2.put(entry.getKey(), tabletMap); for (Entry> tabletRanges : entry.getValue().entrySet()) { Range tabletRange = tabletRanges.getKey().toDataRange(); - List clippedRanges = new ArrayList(); + List clippedRanges = new ArrayList<>(); tabletMap.put(tabletRanges.getKey(), clippedRanges); for (Range range : tabletRanges.getValue()) clippedRanges.add(tabletRange.clip(range)); @@ -294,8 +294,8 @@ public class TabletServerBatchReaderIterator implements Iterator>> binnedRanges = new HashMap>>(); - List allRanges = new ArrayList(); + Map>> binnedRanges = new HashMap<>(); + List allRanges = new ArrayList<>(); for (List ranges : failures.values()) allRanges.addAll(ranges); @@ -338,8 +338,8 @@ public class TabletServerBatchReaderIterator implements Iterator> unscanned = new HashMap>(); - Map> tsFailures = new HashMap>(); + Map> unscanned = new HashMap<>(); + Map> tsFailures = new HashMap<>(); try { TimeoutTracker timeoutTracker = timeoutTrackers.get(tsLocation); if (timeoutTracker == null) { @@ -456,7 +456,7 @@ public class TabletServerBatchReaderIterator implements Iterator> failures = new HashMap>(); + Map> failures = new HashMap<>(); if (timedoutServers.size() > 0) { // go ahead and fail any timed out servers @@ -471,10 +471,10 @@ public class TabletServerBatchReaderIterator implements Iterator locations = new ArrayList(binnedRanges.keySet()); + List locations = new ArrayList<>(binnedRanges.keySet()); Collections.shuffle(locations); - List queryTasks = new ArrayList(); + List queryTasks = new ArrayList<>(); for (final String tsLocation : locations) { @@ -483,13 +483,13 @@ public class TabletServerBatchReaderIterator implements Iterator> tabletSubset = new HashMap>(); + HashMap> tabletSubset = new HashMap<>(); for (Entry> entry : tabletsRanges.entrySet()) { tabletSubset.put(entry.getKey(), entry.getValue()); if (tabletSubset.size() >= maxTabletsPerRequest) { QueryTask queryTask = new QueryTask(tsLocation, tabletSubset, failures, receiver, columns); queryTasks.add(queryTask); - tabletSubset = new HashMap>(); + tabletSubset = new HashMap<>(); } } @@ -512,13 +512,12 @@ public class TabletServerBatchReaderIterator implements Iterator> failures, Map> unscanned, MultiScanResult scanResult) { // translate returned failures, remove them from unscanned, and add them to failures - Map> retFailures = Translator.translate(scanResult.failures, Translators.TKET, new Translator.ListTranslator( - Translators.TRT)); + Map> retFailures = Translator.translate(scanResult.failures, Translators.TKET, new Translator.ListTranslator<>(Translators.TRT)); unscanned.keySet().removeAll(retFailures.keySet()); failures.putAll(retFailures); // translate full scans and remove them from unscanned - HashSet fullScans = new HashSet(Translator.translate(scanResult.fullScans, Translators.TKET)); + HashSet fullScans = new HashSet<>(Translator.translate(scanResult.fullScans, Translators.TKET)); unscanned.keySet().removeAll(fullScans); // remove partial scan from unscanned @@ -606,7 +605,7 @@ public class TabletServerBatchReaderIterator implements Iterator> entry : requested.entrySet()) { - ArrayList ranges = new ArrayList(); + ArrayList ranges = new ArrayList<>(); for (Range range : entry.getValue()) { ranges.add(new Range(range)); } @@ -631,8 +630,7 @@ public class TabletServerBatchReaderIterator implements Iterator> thriftTabletRanges = Translator.translate(requested, Translators.KET, new Translator.ListTranslator( - Translators.RT)); + Map> thriftTabletRanges = Translator.translate(requested, Translators.KET, new Translator.ListTranslator<>(Translators.RT)); InitialMultiScan imsr = client.startMultiScan(Tracer.traceInfo(), context.rpcCreds(), thriftTabletRanges, Translator.translate(columns, Translators.CT), options.serverSideIteratorList, options.serverSideIteratorOptions, ByteBufferUtil.toByteBuffers(authorizations.getAuthorizations()), waitForWrites); @@ -644,9 +642,9 @@ public class TabletServerBatchReaderIterator implements Iterator> entries = new ArrayList>(scanResult.results.size()); + ArrayList> entries = new ArrayList<>(scanResult.results.size()); for (TKeyValue kv : scanResult.results) { - entries.add(new SimpleImmutableEntry(new Key(kv.key), new Value(kv.value))); + entries.add(new SimpleImmutableEntry<>(new Key(kv.key), new Value(kv.value))); } if (entries.size() > 0) @@ -666,9 +664,9 @@ public class TabletServerBatchReaderIterator implements Iterator>(scanResult.results.size()); + entries = new ArrayList<>(scanResult.results.size()); for (TKeyValue kv : scanResult.results) { - entries.add(new SimpleImmutableEntry(new Key(kv.key), new Value(kv.value))); + entries.add(new SimpleImmutableEntry<>(new Key(kv.key), new Value(kv.value))); } if (entries.size() > 0) http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java index a8afa5a..f951bfb 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java @@ -151,8 +151,8 @@ public class TabletServerBatchWriter { // error handling private final Violations violations = new Violations(); - private final Map> authorizationFailures = new HashMap>(); - private final HashSet serverSideErrors = new HashSet(); + private final Map> authorizationFailures = new HashMap<>(); + private final HashSet serverSideErrors = new HashSet<>(); private final FailedMutations failedMutations = new FailedMutations(); private int unknownErrors = 0; private boolean somethingFailed = false; @@ -504,7 +504,7 @@ public class TabletServerBatchWriter { } private void updateAuthorizationFailures(Set keySet, SecurityErrorCode code) { - HashMap map = new HashMap(); + HashMap map = new HashMap<>(); for (KeyExtent ke : keySet) map.put(ke, code); @@ -515,7 +515,7 @@ public class TabletServerBatchWriter { if (authorizationFailures.size() > 0) { // was a table deleted? - HashSet tableIds = new HashSet(); + HashSet tableIds = new HashSet<>(); for (KeyExtent ke : authorizationFailures.keySet()) tableIds.add(ke.getTableId().toString()); @@ -536,7 +536,7 @@ public class TabletServerBatchWriter { for (Entry entry : addition.entrySet()) { Set secs = source.get(entry.getKey()); if (secs == null) { - secs = new HashSet(); + secs = new HashSet<>(); source.put(entry.getKey(), secs); } secs.add(entry.getValue()); @@ -564,9 +564,9 @@ public class TabletServerBatchWriter { private void checkForFailures() throws MutationsRejectedException { if (somethingFailed) { List cvsList = violations.asList(); - HashMap> af = new HashMap>(); + HashMap> af = new HashMap<>(); for (Entry> entry : authorizationFailures.entrySet()) { - HashSet codes = new HashSet(); + HashSet codes = new HashSet<>(); for (SecurityErrorCode sce : entry.getValue()) { codes.add(org.apache.accumulo.core.client.security.SecurityErrorCode.valueOf(sce.name())); @@ -664,10 +664,10 @@ public class TabletServerBatchWriter { private final Map locators; public MutationWriter(int numSendThreads) { - serversMutations = new HashMap>(); - queued = new HashSet(); + serversMutations = new HashMap<>(); + queued = new HashSet<>(); sendThreadPool = new SimpleThreadPool(numSendThreads, this.getClass().getName()); - locators = new HashMap(); + locators = new HashMap<>(); binningThreadPool = new SimpleThreadPool(1, "BinMutations", new SynchronousQueue()); binningThreadPool.setRejectedExecutionHandler(new ThreadPoolExecutor.CallerRunsPolicy()); } @@ -695,7 +695,7 @@ public class TabletServerBatchWriter { List tableMutations = entry.getValue(); if (tableMutations != null) { - ArrayList tableFailures = new ArrayList(); + ArrayList tableFailures = new ArrayList<>(); locator.binMutations(context, tableMutations, binnedMutations, tableFailures); if (tableFailures.size() > 0) { @@ -752,7 +752,7 @@ public class TabletServerBatchWriter { } private void addMutations(MutationSet mutationsToSend) { - Map> binnedMutations = new HashMap>(); + Map> binnedMutations = new HashMap<>(); Span span = Trace.start("binMutations"); try { long t1 = System.currentTimeMillis(); @@ -795,7 +795,7 @@ public class TabletServerBatchWriter { log.trace(String.format("Started sending %,d mutations to %,d tablet servers", count, binnedMutations.keySet().size())); // randomize order of servers - ArrayList servers = new ArrayList(binnedMutations.keySet()); + ArrayList servers = new ArrayList<>(binnedMutations.keySet()); Collections.shuffle(servers); for (String server : servers) @@ -848,7 +848,7 @@ public class TabletServerBatchWriter { long count = 0; - Set tableIds = new TreeSet(); + Set tableIds = new TreeSet<>(); for (Map.Entry> entry : mutationBatch.entrySet()) { count += entry.getValue().size(); tableIds.add(entry.getKey().getTableId()); @@ -896,7 +896,7 @@ public class TabletServerBatchWriter { if (log.isTraceEnabled()) log.trace("failed to send mutations to {} : {}", location, e.getMessage()); - HashSet tables = new HashSet(); + HashSet tables = new HashSet<>(); for (KeyExtent ke : mutationBatch.keySet()) tables.add(ke.getTableId().toString()); @@ -947,7 +947,7 @@ public class TabletServerBatchWriter { long usid = client.startUpdate(tinfo, context.rpcCreds(), DurabilityImpl.toThrift(durability)); - List updates = new ArrayList(); + List updates = new ArrayList<>(); for (Entry> entry : tabMuts.entrySet()) { long size = 0; Iterator iter = entry.getValue().iterator(); @@ -1022,13 +1022,13 @@ public class TabletServerBatchWriter { private int memoryUsed = 0; MutationSet() { - mutations = new HashMap>(); + mutations = new HashMap<>(); } void addMutation(String table, Mutation mutation) { List tabMutList = mutations.get(table); if (tabMutList == null) { - tabMutList = new ArrayList(); + tabMutList = new ArrayList<>(); mutations.put(table, tabMutList); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java index d2fc259..cfb0235 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftScanner.java @@ -71,7 +71,7 @@ import com.google.common.net.HostAndPort; public class ThriftScanner { private static final Logger log = Logger.getLogger(ThriftScanner.class); - public static final Map> serversWaitedForWrites = new EnumMap>(TabletType.class); + public static final Map> serversWaitedForWrites = new EnumMap<>(TabletType.class); static { for (TabletType ttype : TabletType.values()) { @@ -164,7 +164,7 @@ public class ThriftScanner { ; this.authorizations = authorizations; - columns = new ArrayList(fetchedColumns.size()); + columns = new ArrayList<>(fetchedColumns.size()); for (Column column : fetchedColumns) { columns.add(column); } @@ -457,7 +457,7 @@ public class ThriftScanner { if (sr.results.size() > 0 && !scanState.finished) scanState.range = new Range(new Key(sr.results.get(sr.results.size() - 1).key), false, scanState.range.getEndKey(), scanState.range.isEndKeyInclusive()); - List results = new ArrayList(sr.results.size()); + List results = new ArrayList<>(sr.results.size()); for (TKeyValue tkv : sr.results) results.add(new KeyValue(new Key(tkv.key), tkv.value)); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java index ba62cec..682ecbd 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ThriftTransportPool.java @@ -48,10 +48,10 @@ public class ThriftTransportPool { private static final Random random = new Random(); private long killTime = 1000 * 3; - private Map> cache = new HashMap>(); - private Map errorCount = new HashMap(); - private Map errorTime = new HashMap(); - private Set serversWarnedAbout = new HashSet(); + private Map> cache = new HashMap<>(); + private Map errorCount = new HashMap<>(); + private Map errorTime = new HashMap<>(); + private Set serversWarnedAbout = new HashSet<>(); private CountDownLatch closerExitLatch; @@ -95,7 +95,7 @@ public class ThriftTransportPool { private void closeConnections() { while (true) { - ArrayList connectionsToClose = new ArrayList(); + ArrayList connectionsToClose = new ArrayList<>(); synchronized (pool) { for (List ccl : pool.getCache().values()) { @@ -394,7 +394,7 @@ public class ThriftTransportPool { List ccl = getCache().get(cacheKey); if (ccl == null) { - ccl = new LinkedList(); + ccl = new LinkedList<>(); getCache().put(cacheKey, ccl); } @@ -413,10 +413,10 @@ public class ThriftTransportPool { @VisibleForTesting public Pair getAnyTransport(List servers, boolean preferCachedConnection) throws TTransportException { - servers = new ArrayList(servers); + servers = new ArrayList<>(servers); if (preferCachedConnection) { - HashSet serversSet = new HashSet(servers); + HashSet serversSet = new HashSet<>(servers); synchronized (this) { @@ -424,7 +424,7 @@ public class ThriftTransportPool { serversSet.retainAll(getCache().keySet()); if (serversSet.size() > 0) { - ArrayList cachedServers = new ArrayList(serversSet); + ArrayList cachedServers = new ArrayList<>(serversSet); Collections.shuffle(cachedServers, random); for (ThriftTransportKey ttk : cachedServers) { @@ -463,7 +463,7 @@ public class ThriftTransportPool { } try { - return new Pair(ttk.getServer().toString(), createNewTransport(ttk)); + return new Pair<>(ttk.getServer().toString(), createNewTransport(ttk)); } catch (TTransportException tte) { log.debug("Failed to connect to {}", servers.get(index), tte); servers.remove(index); @@ -490,7 +490,7 @@ public class ThriftTransportPool { List ccl = getCache().get(cacheKey); if (ccl == null) { - ccl = new LinkedList(); + ccl = new LinkedList<>(); getCache().put(cacheKey, ccl); } @@ -511,7 +511,7 @@ public class ThriftTransportPool { boolean existInCache = false; CachedTTransport ctsc = (CachedTTransport) tsc; - ArrayList closeList = new ArrayList(); + ArrayList closeList = new ArrayList<>(); synchronized (this) { List ccl = getCache().get(ctsc.getCacheKey()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java index e5141cf..00c43ac 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Translator.java @@ -111,7 +111,7 @@ public abstract class Translator { } public static Map translate(Map input, Translator keyTranslator) { - HashMap output = new HashMap(); + HashMap output = new HashMap<>(); for (Entry entry : input.entrySet()) output.put(keyTranslator.translate(entry.getKey()), entry.getValue()); @@ -120,7 +120,7 @@ public abstract class Translator { } public static Map translate(Map input, Translator keyTranslator, Translator valueTranslator) { - HashMap output = new HashMap(); + HashMap output = new HashMap<>(); for (Entry entry : input.entrySet()) output.put(keyTranslator.translate(entry.getKey()), valueTranslator.translate(entry.getValue())); @@ -129,7 +129,7 @@ public abstract class Translator { } public static List translate(Collection input, Translator translator) { - ArrayList output = new ArrayList(input.size()); + ArrayList output = new ArrayList<>(input.size()); for (IT in : input) output.add(translator.translate(in)); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java index d78cff2..cbdf23e 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java +++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/ListLexicoder.java @@ -66,7 +66,7 @@ public class ListLexicoder extends AbstractLexicoder> implements Le protected List decodeUnchecked(byte[] b, int offset, int len) { byte[][] escapedElements = split(b, offset, len); - ArrayList ret = new ArrayList(escapedElements.length); + ArrayList ret = new ArrayList<>(escapedElements.length); for (byte[] escapedElement : escapedElements) { ret.add(lexicoder.decode(unescape(escapedElement))); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java index 9198b43..7ed2d07 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java +++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/PairLexicoder.java @@ -79,7 +79,7 @@ public class PairLexicoder,B extends Comparable> exte throw new RuntimeException("Data does not have 2 fields, it has " + fields.length); } - return new ComparablePair(firstLexicoder.decode(unescape(fields[0])), secondLexicoder.decode(unescape(fields[1]))); + return new ComparablePair<>(firstLexicoder.decode(unescape(fields[0])), secondLexicoder.decode(unescape(fields[1]))); } } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/lexicoder/impl/ByteUtils.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/impl/ByteUtils.java b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/impl/ByteUtils.java index b168807..0daceaa 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/lexicoder/impl/ByteUtils.java +++ b/core/src/main/java/org/apache/accumulo/core/client/lexicoder/impl/ByteUtils.java @@ -97,7 +97,7 @@ public class ByteUtils { * Splits a byte array by 0x00 */ public static byte[][] split(byte[] data, int dataOffset, int len) { - ArrayList offsets = new ArrayList(); + ArrayList offsets = new ArrayList<>(); for (int i = dataOffset; i < (dataOffset + len); i++) { if (data[i] == 0x00) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java index 86a7adf..967daf6 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java @@ -621,7 +621,7 @@ public abstract class AbstractInputFormat implements InputFormat { validateOptions(job); Random random = new Random(); - LinkedList splits = new LinkedList(); + LinkedList splits = new LinkedList<>(); Map tableConfigs = getInputTableConfigs(job); for (Map.Entry tableConfigEntry : tableConfigs.entrySet()) { String tableName = tableConfigEntry.getKey(); @@ -655,12 +655,12 @@ public abstract class AbstractInputFormat implements InputFormat { List ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges(); if (ranges.isEmpty()) { - ranges = new ArrayList(1); + ranges = new ArrayList<>(1); ranges.add(new Range()); } // get the metadata information for these ranges - Map>> binnedRanges = new HashMap>>(); + Map>> binnedRanges = new HashMap<>(); TabletLocator tl; try { if (tableConfig.isOfflineScan()) { @@ -697,9 +697,9 @@ public abstract class AbstractInputFormat implements InputFormat { HashMap> splitsToAdd = null; if (!autoAdjust) - splitsToAdd = new HashMap>(); + splitsToAdd = new HashMap<>(); - HashMap hostNameCache = new HashMap(); + HashMap hostNameCache = new HashMap<>(); for (Map.Entry>> tserverBin : binnedRanges.entrySet()) { String ip = tserverBin.getKey().split(":", 2)[0]; String location = hostNameCache.get(ip); @@ -712,7 +712,7 @@ public abstract class AbstractInputFormat implements InputFormat { Range ke = extentRanges.getKey().toDataRange(); if (batchScan) { // group ranges by tablet to be read by a BatchScanner - ArrayList clippedRanges = new ArrayList(); + ArrayList clippedRanges = new ArrayList<>(); for (Range r : extentRanges.getValue()) clippedRanges.add(ke.clip(r)); @@ -736,7 +736,7 @@ public abstract class AbstractInputFormat implements InputFormat { // don't divide ranges ArrayList locations = splitsToAdd.get(r); if (locations == null) - locations = new ArrayList(1); + locations = new ArrayList<>(1); locations.add(location); splitsToAdd.put(r, locations); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java index c194cf6..6ad93f3 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java @@ -429,7 +429,7 @@ public class AccumuloOutputFormat implements OutputFormat { if (simulate) log.info("Simulating output only. No writes to tables will occur"); - this.bws = new HashMap(); + this.bws = new HashMap<>(); String tname = getDefaultTableName(job); this.defaultTableName = (tname == null) ? null : new Text(tname); @@ -543,11 +543,11 @@ public class AccumuloOutputFormat implements OutputFormat { mtbw.close(); } catch (MutationsRejectedException e) { if (e.getSecurityErrorCodes().size() >= 0) { - HashMap> tables = new HashMap>(); + HashMap> tables = new HashMap<>(); for (Entry> ke : e.getSecurityErrorCodes().entrySet()) { Set secCodes = tables.get(ke.getKey().getTableId().toString()); if (secCodes == null) { - secCodes = new HashSet(); + secCodes = new HashSet<>(); tables.put(ke.getKey().getTableId().toString(), secCodes); } secCodes.addAll(ke.getValue()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java index 6f257ff..ab60327 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloRowInputFormat.java @@ -78,7 +78,7 @@ public class AccumuloRowInputFormat extends InputFormatBase> createValue() { - return new PeekingIterator>(); + return new PeekingIterator<>(); } }; recordReader.initialize(split, job); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java index 2575fe5..83aa269 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AbstractInputFormat.java @@ -667,7 +667,7 @@ public abstract class AbstractInputFormat extends InputFormat { log.setLevel(logLevel); validateOptions(context); Random random = new Random(); - LinkedList splits = new LinkedList(); + LinkedList splits = new LinkedList<>(); Map tableConfigs = getInputTableConfigs(context); for (Map.Entry tableConfigEntry : tableConfigs.entrySet()) { @@ -702,12 +702,12 @@ public abstract class AbstractInputFormat extends InputFormat { List ranges = autoAdjust ? Range.mergeOverlapping(tableConfig.getRanges()) : tableConfig.getRanges(); if (ranges.isEmpty()) { - ranges = new ArrayList(1); + ranges = new ArrayList<>(1); ranges.add(new Range()); } // get the metadata information for these ranges - Map>> binnedRanges = new HashMap>>(); + Map>> binnedRanges = new HashMap<>(); TabletLocator tl; try { if (tableConfig.isOfflineScan()) { @@ -747,9 +747,9 @@ public abstract class AbstractInputFormat extends InputFormat { HashMap> splitsToAdd = null; if (!autoAdjust) - splitsToAdd = new HashMap>(); + splitsToAdd = new HashMap<>(); - HashMap hostNameCache = new HashMap(); + HashMap hostNameCache = new HashMap<>(); for (Map.Entry>> tserverBin : binnedRanges.entrySet()) { String ip = tserverBin.getKey().split(":", 2)[0]; String location = hostNameCache.get(ip); @@ -762,7 +762,7 @@ public abstract class AbstractInputFormat extends InputFormat { Range ke = extentRanges.getKey().toDataRange(); if (batchScan) { // group ranges by tablet to be read by a BatchScanner - ArrayList clippedRanges = new ArrayList(); + ArrayList clippedRanges = new ArrayList<>(); for (Range r : extentRanges.getValue()) clippedRanges.add(ke.clip(r)); BatchInputSplit split = new BatchInputSplit(tableName, tableId, clippedRanges, new String[] {location}); @@ -785,7 +785,7 @@ public abstract class AbstractInputFormat extends InputFormat { // don't divide ranges ArrayList locations = splitsToAdd.get(r); if (locations == null) - locations = new ArrayList(1); + locations = new ArrayList<>(1); locations.add(location); splitsToAdd.put(r, locations); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java index 4cb46a3..e736a0f 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloOutputFormat.java @@ -429,7 +429,7 @@ public class AccumuloOutputFormat extends OutputFormat { if (simulate) log.info("Simulating output only. No writes to tables will occur"); - this.bws = new HashMap(); + this.bws = new HashMap<>(); String tname = getDefaultTableName(context); this.defaultTableName = (tname == null) ? null : new Text(tname); @@ -543,11 +543,11 @@ public class AccumuloOutputFormat extends OutputFormat { mtbw.close(); } catch (MutationsRejectedException e) { if (e.getSecurityErrorCodes().size() >= 0) { - HashMap> tables = new HashMap>(); + HashMap> tables = new HashMap<>(); for (Entry> ke : e.getSecurityErrorCodes().entrySet()) { Set secCodes = tables.get(ke.getKey().getTableId().toString()); if (secCodes == null) { - secCodes = new HashSet(); + secCodes = new HashSet<>(); tables.put(ke.getKey().getTableId().toString(), secCodes); } secCodes.addAll(ke.getValue()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java index 77081bf..53a45ca 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/AccumuloRowInputFormat.java @@ -68,7 +68,7 @@ public class AccumuloRowInputFormat extends InputFormatBase>(rowIterator.next()); + currentV = new PeekingIterator<>(rowIterator.next()); numKeysRead = rowIterator.getKVCount(); currentKey = currentV.peek().getKey(); currentK = new Text(currentKey.getRow()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java index 257f6c9..505c75a 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/InputTableConfig.java @@ -283,13 +283,13 @@ public class InputTableConfig implements Writable { // load iterators long iterSize = dataInput.readInt(); if (iterSize > 0) - iterators = new ArrayList(); + iterators = new ArrayList<>(); for (int i = 0; i < iterSize; i++) iterators.add(new IteratorSetting(dataInput)); // load ranges long rangeSize = dataInput.readInt(); if (rangeSize > 0) - ranges = new ArrayList(); + ranges = new ArrayList<>(); for (int i = 0; i < rangeSize; i++) { Range range = new Range(); range.readFields(dataInput); @@ -298,7 +298,7 @@ public class InputTableConfig implements Writable { // load columns long columnSize = dataInput.readInt(); if (columnSize > 0) - columns = new HashSet>(); + columns = new HashSet<>(); for (int i = 0; i < columnSize; i++) { long numPairs = dataInput.readInt(); Text colFam = new Text(); @@ -308,7 +308,7 @@ public class InputTableConfig implements Writable { } else if (numPairs == 2) { Text colQual = new Text(); colQual.readFields(dataInput); - columns.add(new Pair(colFam, colQual)); + columns.add(new Pair<>(colFam, colQual)); } } autoAdjustRanges = dataInput.readBoolean(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/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 e337977..41ace4f 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 @@ -157,7 +157,7 @@ public class RangeInputSplit extends InputSplit implements Writable { if (in.readBoolean()) { int numColumns = in.readInt(); - List columns = new ArrayList(numColumns); + List columns = new ArrayList<>(numColumns); for (int i = 0; i < numColumns; i++) { columns.add(in.readUTF()); } @@ -206,7 +206,7 @@ public class RangeInputSplit extends InputSplit implements Writable { if (in.readBoolean()) { int numIterators = in.readInt(); - iterators = new ArrayList(numIterators); + iterators = new ArrayList<>(numIterators); for (int i = 0; i < numIterators; i++) { iterators.add(new IteratorSetting(in)); } @@ -455,7 +455,7 @@ public class RangeInputSplit extends InputSplit implements Writable { } public void setFetchedColumns(Collection> fetchedColumns) { - this.fetchedColumns = new HashSet>(); + this.fetchedColumns = new HashSet<>(); for (Pair columns : fetchedColumns) { this.fetchedColumns.add(columns); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplit.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplit.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplit.java index 04875ac..2965788 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplit.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/impl/BatchInputSplit.java @@ -107,7 +107,7 @@ public class BatchInputSplit extends RangeInputSplit { super.readFields(in); int numRanges = in.readInt(); - ranges = new ArrayList(numRanges); + ranges = new ArrayList<>(numRanges); for (int i = 0; i < numRanges; ++i) { Range r = new Range(); r.readFields(in); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java index 0e640b4..eaf6f6d 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mapreduce/lib/impl/InputConfigurator.java @@ -176,7 +176,7 @@ public class InputConfigurator extends ConfiguratorBase { public static void setRanges(Class implementingClass, Configuration conf, Collection ranges) { checkArgument(ranges != null, "ranges is null"); - ArrayList rangeStrings = new ArrayList(ranges.size()); + ArrayList rangeStrings = new ArrayList<>(ranges.size()); try { for (Range r : ranges) { ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -205,7 +205,7 @@ public class InputConfigurator extends ConfiguratorBase { public static List getRanges(Class implementingClass, Configuration conf) throws IOException { Collection encodedRanges = conf.getStringCollection(enumToConfKey(implementingClass, ScanOpts.RANGES)); - List ranges = new ArrayList(); + List ranges = new ArrayList<>(); for (String rangeString : encodedRanges) { ByteArrayInputStream bais = new ByteArrayInputStream(Base64.decodeBase64(rangeString.getBytes(UTF_8))); Range range = new Range(); @@ -231,11 +231,11 @@ public class InputConfigurator extends ConfiguratorBase { // If no iterators are present, return an empty list if (iterators == null || iterators.isEmpty()) - return new ArrayList(); + return new ArrayList<>(); // Compose the set of iterators encoded in the job configuration StringTokenizer tokens = new StringTokenizer(iterators, StringUtils.COMMA_STR); - List list = new ArrayList(); + List list = new ArrayList<>(); try { while (tokens.hasMoreTokens()) { String itstring = tokens.nextToken(); @@ -271,7 +271,7 @@ public class InputConfigurator extends ConfiguratorBase { public static String[] serializeColumns(Collection> columnFamilyColumnQualifierPairs) { checkArgument(columnFamilyColumnQualifierPairs != null, "columnFamilyColumnQualifierPairs is null"); - ArrayList columnStrings = new ArrayList(columnFamilyColumnQualifierPairs.size()); + ArrayList columnStrings = new ArrayList<>(columnFamilyColumnQualifierPairs.size()); for (Pair column : columnFamilyColumnQualifierPairs) { if (column.getFirst() == null) @@ -300,7 +300,7 @@ public class InputConfigurator extends ConfiguratorBase { public static Set> getFetchedColumns(Class implementingClass, Configuration conf) { checkArgument(conf != null, "conf is null"); String confValue = conf.get(enumToConfKey(implementingClass, ScanOpts.COLUMNS)); - List serialized = new ArrayList(); + List serialized = new ArrayList<>(); if (confValue != null) { // Split and include any trailing empty strings to allow empty column families for (String val : confValue.split(",", -1)) { @@ -311,7 +311,7 @@ public class InputConfigurator extends ConfiguratorBase { } public static Set> deserializeFetchedColumns(Collection serialized) { - Set> columns = new HashSet>(); + Set> columns = new HashSet<>(); if (null == serialized) { return columns; @@ -321,7 +321,7 @@ public class InputConfigurator extends ConfiguratorBase { int idx = col.indexOf(":"); Text cf = new Text(idx < 0 ? Base64.decodeBase64(col.getBytes(UTF_8)) : Base64.decodeBase64(col.substring(0, idx).getBytes(UTF_8))); Text cq = idx < 0 ? null : new Text(Base64.decodeBase64(col.substring(idx + 1).getBytes(UTF_8))); - columns.add(new Pair(cf, cq)); + columns.add(new Pair<>(cf, cq)); } return columns; } @@ -589,7 +589,7 @@ public class InputConfigurator extends ConfiguratorBase { * @since 1.6.0 */ public static Map getInputTableConfigs(Class implementingClass, Configuration conf) { - Map configs = new HashMap(); + Map configs = new HashMap<>(); Map.Entry defaultConfig = getDefaultInputTableConfig(implementingClass, conf); if (defaultConfig != null) configs.put(defaultConfig.getKey(), defaultConfig.getValue()); @@ -814,7 +814,7 @@ public class InputConfigurator extends ConfiguratorBase { public static Map>> binOffline(String tableId, List ranges, Instance instance, Connector conn) throws AccumuloException, TableNotFoundException { - Map>> binnedRanges = new HashMap>>(); + Map>> binnedRanges = new HashMap<>(); if (Tables.getTableState(instance, tableId) != TableState.OFFLINE) { Tables.clearCache(instance); @@ -879,13 +879,13 @@ public class InputConfigurator extends ConfiguratorBase { Map> tabletRanges = binnedRanges.get(last); if (tabletRanges == null) { - tabletRanges = new HashMap>(); + tabletRanges = new HashMap<>(); binnedRanges.put(last, tabletRanges); } List rangeList = tabletRanges.get(extent); if (rangeList == null) { - rangeList = new ArrayList(); + rangeList = new ArrayList<>(); tabletRanges.put(extent, rangeList); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/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 c0c0097..fa80831 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 @@ -89,7 +89,7 @@ public class RangePartitioner extends Partitioner implements Conf if (cf != null) { for (Path path : cf) { if (path.toUri().getPath().endsWith(cutFileName.substring(cutFileName.lastIndexOf('/')))) { - TreeSet cutPoints = new TreeSet(); + TreeSet cutPoints = new TreeSet<>(); Scanner in = new Scanner(new BufferedReader(new InputStreamReader(new FileInputStream(path.toString()), UTF_8))); try { while (in.hasNextLine()) http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java index e1ca768..eaa2e4e 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockAccumulo.java @@ -41,10 +41,10 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.io.Text; public class MockAccumulo { - final Map tables = new HashMap(); - final Map namespaces = new HashMap(); - final Map systemProperties = new HashMap(); - Map users = new HashMap(); + final Map tables = new HashMap<>(); + final Map namespaces = new HashMap<>(); + final Map systemProperties = new HashMap<>(); + Map users = new HashMap<>(); final FileSystem fs; final AtomicInteger tableIdCounter = new AtomicInteger(0); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java index 4034271..5b2b8e9 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockBatchScanner.java @@ -46,7 +46,7 @@ public class MockBatchScanner extends MockScannerBase implements BatchScanner { throw new IllegalArgumentException("ranges must be non null and contain at least 1 range"); } - this.ranges = new ArrayList(ranges); + this.ranges = new ArrayList<>(ranges); } @SuppressWarnings("unchecked") http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java index 57cd5ee..839b478 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java @@ -54,7 +54,7 @@ import org.apache.hadoop.io.Text; public class MockInstance implements Instance { static final String genericAddress = "localhost:1234"; - static final Map instances = new HashMap(); + static final Map instances = new HashMap<>(); MockAccumulo acu; String instanceName; http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java index c1acc04..05e171d 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstanceOperations.java @@ -59,12 +59,12 @@ class MockInstanceOperations implements InstanceOperations { @Override public List getTabletServers() { - return new ArrayList(); + return new ArrayList<>(); } @Override public List getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException { - return new ArrayList(); + return new ArrayList<>(); } @Override @@ -80,7 +80,7 @@ class MockInstanceOperations implements InstanceOperations { @Override public List getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException { - return new ArrayList(); + return new ArrayList<>(); } @Override http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java index 9cc3dfb..0e04854 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockMultiTableBatchWriter.java @@ -32,7 +32,7 @@ public class MockMultiTableBatchWriter implements MultiTableBatchWriter { public MockMultiTableBatchWriter(MockAccumulo acu) { this.acu = acu; - bws = new HashMap(); + bws = new HashMap<>(); } @Override http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java index 955564f..32f0a24 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespace.java @@ -30,10 +30,10 @@ import org.apache.accumulo.core.security.NamespacePermission; public class MockNamespace { final HashMap settings; - Map> userPermissions = new HashMap>(); + Map> userPermissions = new HashMap<>(); public MockNamespace() { - settings = new HashMap(); + settings = new HashMap<>(); for (Entry entry : AccumuloConfiguration.getDefaultConfiguration()) { String key = entry.getKey(); if (key.startsWith(Property.TABLE_PREFIX.getKey())) { @@ -43,7 +43,7 @@ public class MockNamespace { } public List getTables(MockAccumulo acu) { - List l = new LinkedList(); + List l = new LinkedList<>(); for (String t : acu.tables.keySet()) { if (acu.tables.get(t).getNamespace().equals(this)) { l.add(t); http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java index 004124d..fc8d73b 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockNamespaceOperations.java @@ -48,7 +48,7 @@ class MockNamespaceOperations extends NamespaceOperationsHelper { @Override public SortedSet list() { - return new TreeSet(acu.namespaces.keySet()); + return new TreeSet<>(acu.namespaces.keySet()); } @Override @@ -112,7 +112,7 @@ class MockNamespaceOperations extends NamespaceOperationsHelper { @Override public Map namespaceIdMap() { - Map result = new HashMap(); + Map result = new HashMap<>(); for (String table : acu.tables.keySet()) { result.put(table, table); } http://git-wip-us.apache.org/repos/asf/accumulo/blob/95f42858/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java index 3c746e1..0a31bf2 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockScannerBase.java @@ -54,7 +54,7 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase { } static HashSet createColumnBSS(Collection columns) { - HashSet columnSet = new HashSet(); + HashSet columnSet = new HashSet<>(); for (Column c : columns) { columnSet.add(new ArrayByteSequence(c.getColumnFamily())); } @@ -89,7 +89,7 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase { return false; } - private ArrayList> topLevelIterators = new ArrayList>(); + private ArrayList> topLevelIterators = new ArrayList<>(); @Override public void registerSideChannel(SortedKeyValueIterator iter) { @@ -104,7 +104,7 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase { SortedKeyValueIterator getTopLevelIterator(SortedKeyValueIterator iter) { if (topLevelIterators.isEmpty()) return iter; - ArrayList> allIters = new ArrayList>(topLevelIterators); + ArrayList> allIters = new ArrayList<>(topLevelIterators); allIters.add(iter); return new MultiIterator(allIters, false); } @@ -113,7 +113,7 @@ public class MockScannerBase extends ScannerOptions implements ScannerBase { public SortedKeyValueIterator createFilter(SortedKeyValueIterator inner) throws IOException { byte[] defaultLabels = {}; inner = new ColumnFamilySkippingIterator(new DeletingIterator(inner, false)); - ColumnQualifierFilter cqf = new ColumnQualifierFilter(inner, new HashSet(fetchedColumns)); + ColumnQualifierFilter cqf = new ColumnQualifierFilter(inner, new HashSet<>(fetchedColumns)); VisibilityFilter vf = new VisibilityFilter(cqf, auths, defaultLabels); AccumuloConfiguration conf = new MockConfiguration(table.settings); MockIteratorEnvironment iterEnv = new MockIteratorEnvironment(auths);