You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@accumulo.apache.org by GitBox <gi...@apache.org> on 2018/12/18 01:01:44 UTC

[GitHub] ctubbsii closed pull request #835: Close some unclosed objects causing resource leaks

ctubbsii closed pull request #835: Close some unclosed objects causing resource leaks
URL: https://github.com/apache/accumulo/pull/835
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 22505b916f..8c2071492a 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
@@ -480,121 +480,123 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException {
       baseSplit = (org.apache.accumulo.core.client.mapreduce.RangeInputSplit) inSplit;
       log.debug("Initializing input split: " + baseSplit);
 
-      ClientContext context = new ClientContext(getClientProperties(job));
-      Authorizations authorizations = getScanAuthorizations(job);
-      String classLoaderContext = getClassLoaderContext(job);
-      String table = baseSplit.getTableName();
-
-      // in case the table name changed, we can still use the previous name for terms of
-      // configuration, but the scanner will use the table id resolved at job setup time
-      InputTableConfig tableConfig = getInputTableConfig(job, baseSplit.getTableName());
-
-      log.debug("Created client with user: " + context.whoami());
-      log.debug("Creating scanner for table: " + table);
-      log.debug("Authorizations are: " + authorizations);
-
-      if (baseSplit instanceof BatchInputSplit) {
-        BatchScanner scanner;
-        BatchInputSplit multiRangeSplit = (BatchInputSplit) baseSplit;
-
-        try {
-          // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
-          // will not span tablets
-          int scanThreads = 1;
-          scanner = context.createBatchScanner(baseSplit.getTableName(), authorizations,
-              scanThreads);
-          setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
-          if (null != classLoaderContext) {
-            scanner.setClassLoaderContext(classLoaderContext);
+      try (ClientContext context = new ClientContext(getClientProperties(job))) {
+        Authorizations authorizations = getScanAuthorizations(job);
+        String classLoaderContext = getClassLoaderContext(job);
+        String table = baseSplit.getTableName();
+
+        // in case the table name changed, we can still use the previous name for terms of
+        // configuration, but the scanner will use the table id resolved at job setup time
+        InputTableConfig tableConfig = getInputTableConfig(job, baseSplit.getTableName());
+
+        log.debug("Created client with user: " + context.whoami());
+        log.debug("Creating scanner for table: " + table);
+        log.debug("Authorizations are: " + authorizations);
+
+        if (baseSplit instanceof BatchInputSplit) {
+          BatchScanner scanner;
+          BatchInputSplit multiRangeSplit = (BatchInputSplit) baseSplit;
+
+          try {
+            // Note: BatchScanner will use at most one thread per tablet, currently BatchInputSplit
+            // will not span tablets
+            int scanThreads = 1;
+            scanner = context.createBatchScanner(baseSplit.getTableName(), authorizations,
+                scanThreads);
+            setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
+            if (null != classLoaderContext) {
+              scanner.setClassLoaderContext(classLoaderContext);
+            }
+          } catch (Exception e) {
+            throw new IOException(e);
           }
-        } catch (Exception e) {
-          throw new IOException(e);
-        }
 
-        scanner.setRanges(multiRangeSplit.getRanges());
-        scannerBase = scanner;
+          scanner.setRanges(multiRangeSplit.getRanges());
+          scannerBase = scanner;
 
-      } else if (baseSplit instanceof RangeInputSplit) {
-        split = (RangeInputSplit) baseSplit;
-        Boolean isOffline = baseSplit.isOffline();
-        if (null == isOffline) {
-          isOffline = tableConfig.isOfflineScan();
-        }
+        } else if (baseSplit instanceof RangeInputSplit) {
+          split = (RangeInputSplit) baseSplit;
+          Boolean isOffline = baseSplit.isOffline();
+          if (null == isOffline) {
+            isOffline = tableConfig.isOfflineScan();
+          }
 
-        Boolean isIsolated = baseSplit.isIsolatedScan();
-        if (null == isIsolated) {
-          isIsolated = tableConfig.shouldUseIsolatedScanners();
-        }
+          Boolean isIsolated = baseSplit.isIsolatedScan();
+          if (null == isIsolated) {
+            isIsolated = tableConfig.shouldUseIsolatedScanners();
+          }
 
-        Boolean usesLocalIterators = baseSplit.usesLocalIterators();
-        if (null == usesLocalIterators) {
-          usesLocalIterators = tableConfig.shouldUseLocalIterators();
-        }
+          Boolean usesLocalIterators = baseSplit.usesLocalIterators();
+          if (null == usesLocalIterators) {
+            usesLocalIterators = tableConfig.shouldUseLocalIterators();
+          }
 
-        Scanner scanner;
+          Scanner scanner;
 
-        try {
-          if (isOffline) {
-            scanner = new OfflineScanner(context, Table.ID.of(baseSplit.getTableId()),
-                authorizations);
-          } else {
-            scanner = new ScannerImpl(context, Table.ID.of(baseSplit.getTableId()), authorizations);
-          }
-          if (isIsolated) {
-            log.info("Creating isolated scanner");
-            scanner = new IsolatedScanner(scanner);
-          }
-          if (usesLocalIterators) {
-            log.info("Using local iterators");
-            scanner = new ClientSideIteratorScanner(scanner);
+          try {
+            if (isOffline) {
+              scanner = new OfflineScanner(context, Table.ID.of(baseSplit.getTableId()),
+                  authorizations);
+            } else {
+              scanner = new ScannerImpl(context, Table.ID.of(baseSplit.getTableId()),
+                  authorizations);
+            }
+            if (isIsolated) {
+              log.info("Creating isolated scanner");
+              scanner = new IsolatedScanner(scanner);
+            }
+            if (usesLocalIterators) {
+              log.info("Using local iterators");
+              scanner = new ClientSideIteratorScanner(scanner);
+            }
+            setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
+          } catch (Exception e) {
+            throw new IOException(e);
           }
-          setupIterators(job, scanner, baseSplit.getTableName(), baseSplit);
-        } catch (Exception e) {
-          throw new IOException(e);
+
+          scanner.setRange(baseSplit.getRange());
+          scannerBase = scanner;
+        } else {
+          throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
         }
 
-        scanner.setRange(baseSplit.getRange());
-        scannerBase = scanner;
-      } else {
-        throw new IllegalArgumentException("Can not initialize from " + baseSplit.getClass());
-      }
+        Collection<Pair<Text,Text>> columns = baseSplit.getFetchedColumns();
+        if (null == columns) {
+          columns = tableConfig.getFetchedColumns();
+        }
 
-      Collection<Pair<Text,Text>> columns = baseSplit.getFetchedColumns();
-      if (null == columns) {
-        columns = tableConfig.getFetchedColumns();
-      }
+        // setup a scanner within the bounds of this split
+        for (Pair<Text,Text> c : columns) {
+          if (c.getSecond() != null) {
+            log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
+            scannerBase.fetchColumn(c.getFirst(), c.getSecond());
+          } else {
+            log.debug("Fetching column family " + c.getFirst());
+            scannerBase.fetchColumnFamily(c.getFirst());
+          }
+        }
 
-      // setup a scanner within the bounds of this split
-      for (Pair<Text,Text> c : columns) {
-        if (c.getSecond() != null) {
-          log.debug("Fetching column " + c.getFirst() + ":" + c.getSecond());
-          scannerBase.fetchColumn(c.getFirst(), c.getSecond());
-        } else {
-          log.debug("Fetching column family " + c.getFirst());
-          scannerBase.fetchColumnFamily(c.getFirst());
+        SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
+        if (null == samplerConfig) {
+          samplerConfig = tableConfig.getSamplerConfiguration();
         }
-      }
 
-      SamplerConfiguration samplerConfig = baseSplit.getSamplerConfiguration();
-      if (null == samplerConfig) {
-        samplerConfig = tableConfig.getSamplerConfiguration();
-      }
+        if (samplerConfig != null) {
+          scannerBase.setSamplerConfiguration(samplerConfig);
+        }
 
-      if (samplerConfig != null) {
-        scannerBase.setSamplerConfiguration(samplerConfig);
-      }
+        Map<String,String> executionHints = baseSplit.getExecutionHints();
+        if (executionHints == null || executionHints.size() == 0) {
+          executionHints = tableConfig.getExecutionHints();
+        }
 
-      Map<String,String> executionHints = baseSplit.getExecutionHints();
-      if (executionHints == null || executionHints.size() == 0) {
-        executionHints = tableConfig.getExecutionHints();
-      }
+        if (executionHints != null) {
+          scannerBase.setExecutionHints(executionHints);
+        }
 
-      if (executionHints != null) {
-        scannerBase.setExecutionHints(executionHints);
+        scannerIterator = scannerBase.iterator();
+        numKeysRead = 0;
       }
-
-      scannerIterator = scannerBase.iterator();
-      numKeysRead = 0;
     }
 
     @Override
diff --git a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
index 34e4c8c7b1..d55549f31a 100644
--- a/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
+++ b/minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloClusterImpl.java
@@ -844,7 +844,7 @@ public MasterMonitorInfo getMasterMonitorInfo()
     while (true) {
       try (AccumuloClient c = Accumulo.newClient().from(getClientProperties()).build()) {
         client = MasterClient.getConnectionWithRetry((ClientContext) c);
-        return client.getMasterStats(Tracer.traceInfo(), ((ClientContext)c).rpcCreds());
+        return client.getMasterStats(Tracer.traceInfo(), ((ClientContext) c).rpcCreds());
       } catch (ThriftSecurityException exception) {
         throw new AccumuloSecurityException(exception);
       } catch (ThriftNotActiveServiceException e) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java b/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
index 00932d3766..ff5402c349 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/conf/ConfigSanityCheck.java
@@ -26,8 +26,9 @@
 public class ConfigSanityCheck implements KeywordExecutable {
 
   public static void main(String[] args) {
-    ServerContext context = new ServerContext(new SiteConfiguration());
-    context.getServerConfFactory().getSystemConfiguration();
+    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+      context.getServerConfFactory().getSystemConfiguration();
+    }
   }
 
   @Override
diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
index 8499f56b04..5ef502b6b3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java
@@ -393,70 +393,71 @@ private boolean initialize(SiteConfiguration siteConfig, Opts opts, String insta
       return false;
     }
 
-    final ServerContext context = new ServerContext(siteConfig);
+    try (ServerContext context = new ServerContext(siteConfig)) {
 
-    // When we're using Kerberos authentication, we need valid credentials to perform
-    // initialization. If the user provided some, use them.
-    // If they did not, fall back to the credentials present in accumulo.properties that the servers
-    // will use themselves.
-    try {
-      final SiteConfiguration siteConf = context.getServerConfFactory().getSiteConfiguration();
-      if (siteConf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
-        final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
-        // We don't have any valid creds to talk to HDFS
-        if (!ugi.hasKerberosCredentials()) {
-          final String accumuloKeytab = siteConf.get(Property.GENERAL_KERBEROS_KEYTAB),
-              accumuloPrincipal = siteConf.get(Property.GENERAL_KERBEROS_PRINCIPAL);
-
-          // Fail if the site configuration doesn't contain appropriate credentials to login as
-          // servers
-          if (StringUtils.isBlank(accumuloKeytab) || StringUtils.isBlank(accumuloPrincipal)) {
-            log.error("FATAL: No Kerberos credentials provided, and Accumulo is"
-                + " not properly configured for server login");
-            return false;
-          }
+      // When we're using Kerberos authentication, we need valid credentials to perform
+      // initialization. If the user provided some, use them.
+      // If they did not, fall back to the credentials present in accumulo.properties that the
+      // servers will use themselves.
+      try {
+        final SiteConfiguration siteConf = context.getServerConfFactory().getSiteConfiguration();
+        if (siteConf.getBoolean(Property.INSTANCE_RPC_SASL_ENABLED)) {
+          final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
+          // We don't have any valid creds to talk to HDFS
+          if (!ugi.hasKerberosCredentials()) {
+            final String accumuloKeytab = siteConf.get(Property.GENERAL_KERBEROS_KEYTAB),
+                accumuloPrincipal = siteConf.get(Property.GENERAL_KERBEROS_PRINCIPAL);
+
+            // Fail if the site configuration doesn't contain appropriate credentials to login as
+            // servers
+            if (StringUtils.isBlank(accumuloKeytab) || StringUtils.isBlank(accumuloPrincipal)) {
+              log.error("FATAL: No Kerberos credentials provided, and Accumulo is"
+                  + " not properly configured for server login");
+              return false;
+            }
 
-          log.info("Logging in as {} with {}", accumuloPrincipal, accumuloKeytab);
+            log.info("Logging in as {} with {}", accumuloPrincipal, accumuloKeytab);
 
-          // Login using the keytab as the 'accumulo' user
-          UserGroupInformation.loginUserFromKeytab(accumuloPrincipal, accumuloKeytab);
+            // Login using the keytab as the 'accumulo' user
+            UserGroupInformation.loginUserFromKeytab(accumuloPrincipal, accumuloKeytab);
+          }
         }
+      } catch (IOException e) {
+        log.error("FATAL: Failed to get the Kerberos user", e);
+        return false;
       }
-    } catch (IOException e) {
-      log.error("FATAL: Failed to get the Kerberos user", e);
-      return false;
-    }
-
-    try {
-      initSecurity(context, opts, uuid.toString(), rootUser);
-    } catch (Exception e) {
-      log.error("FATAL: Failed to initialize security", e);
-      return false;
-    }
 
-    if (opts.uploadAccumuloProps) {
       try {
-        log.info("Uploading properties in accumulo.properties to Zookeeper."
-            + " Properties that cannot be set in Zookeeper will be skipped:");
-        Map<String,String> entries = new TreeMap<>();
-        siteConfig.getProperties(entries, x -> true, false);
-        for (Map.Entry<String,String> entry : entries.entrySet()) {
-          String key = entry.getKey();
-          String value = entry.getValue();
-          if (Property.isValidZooPropertyKey(key)) {
-            SystemPropUtil.setSystemProperty(context, key, value);
-            log.info("Uploaded - {} = {}", key, Property.isSensitive(key) ? "<hidden>" : value);
-          } else {
-            log.info("Skipped - {} = {}", key, Property.isSensitive(key) ? "<hidden>" : value);
-          }
-        }
+        initSecurity(context, opts, uuid.toString(), rootUser);
       } catch (Exception e) {
-        log.error("FATAL: Failed to upload accumulo.properties to Zookeeper", e);
+        log.error("FATAL: Failed to initialize security", e);
         return false;
       }
-    }
 
-    return true;
+      if (opts.uploadAccumuloProps) {
+        try {
+          log.info("Uploading properties in accumulo.properties to Zookeeper."
+              + " Properties that cannot be set in Zookeeper will be skipped:");
+          Map<String,String> entries = new TreeMap<>();
+          siteConfig.getProperties(entries, x -> true, false);
+          for (Map.Entry<String,String> entry : entries.entrySet()) {
+            String key = entry.getKey();
+            String value = entry.getValue();
+            if (Property.isValidZooPropertyKey(key)) {
+              SystemPropUtil.setSystemProperty(context, key, value);
+              log.info("Uploaded - {} = {}", key, Property.isSensitive(key) ? "<hidden>" : value);
+            } else {
+              log.info("Skipped - {} = {}", key, Property.isSensitive(key) ? "<hidden>" : value);
+            }
+          }
+        } catch (Exception e) {
+          log.error("FATAL: Failed to upload accumulo.properties to Zookeeper", e);
+          return false;
+        }
+      }
+
+      return true;
+    }
   }
 
   private static boolean zookeeperAvailable() {
@@ -938,23 +939,24 @@ public void execute(final String[] args) {
 
       if (opts.resetSecurity) {
         log.info("Resetting security on accumulo.");
-        ServerContext context = new ServerContext(siteConfig);
-        if (isInitialized(fs, siteConfig)) {
-          if (!opts.forceResetSecurity) {
-            ConsoleReader c = getConsoleReader();
-            String userEnteredName = c.readLine("WARNING: This will remove all"
-                + " users from Accumulo! If you wish to proceed enter the instance" + " name: ");
-            if (userEnteredName != null && !context.getInstanceName().equals(userEnteredName)) {
-              log.error("Aborted reset security: Instance name did not match current instance.");
-              return;
+        try (ServerContext context = new ServerContext(siteConfig)) {
+          if (isInitialized(fs, siteConfig)) {
+            if (!opts.forceResetSecurity) {
+              ConsoleReader c = getConsoleReader();
+              String userEnteredName = c.readLine("WARNING: This will remove all"
+                  + " users from Accumulo! If you wish to proceed enter the instance" + " name: ");
+              if (userEnteredName != null && !context.getInstanceName().equals(userEnteredName)) {
+                log.error("Aborted reset security: Instance name did not match current instance.");
+                return;
+              }
             }
-          }
 
-          final String rootUser = getRootUserName(siteConfig, opts);
-          opts.rootpass = getRootPassword(siteConfig, opts, rootUser);
-          initSecurity(context, opts, context.getInstanceID(), rootUser);
-        } else {
-          log.error("FATAL: Attempted to reset security on accumulo before it was initialized");
+            final String rootUser = getRootUserName(siteConfig, opts);
+            opts.rootpass = getRootPassword(siteConfig, opts, rootUser);
+            initSecurity(context, opts, context.getInstanceID(), rootUser);
+          } else {
+            log.error("FATAL: Attempted to reset security on accumulo before it was initialized");
+          }
         }
       }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java b/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
index eef44b73eb..b2ce11b856 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/CleanZookeeper.java
@@ -50,15 +50,14 @@ public static void main(String[] args) {
     Opts opts = new Opts();
     opts.parseArgs(CleanZookeeper.class.getName(), args);
 
-    ServerContext context = new ServerContext(new SiteConfiguration());
+    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
 
-    String root = Constants.ZROOT;
-    IZooReaderWriter zk = context.getZooReaderWriter();
-    if (opts.auth != null) {
-      zk.getZooKeeper().addAuthInfo("digest", ("accumulo:" + opts.auth).getBytes(UTF_8));
-    }
+      String root = Constants.ZROOT;
+      IZooReaderWriter zk = context.getZooReaderWriter();
+      if (opts.auth != null) {
+        zk.getZooKeeper().addAuthInfo("digest", ("accumulo:" + opts.auth).getBytes(UTF_8));
+      }
 
-    try {
       for (String child : zk.getChildren(root)) {
         if (Constants.ZINSTANCES.equals("/" + child)) {
           for (String instanceName : zk.getChildren(root + Constants.ZINSTANCES)) {
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java b/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
index c6bf4fe792..13d9c32be1 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/LoginProperties.java
@@ -47,29 +47,32 @@ public String description() {
 
   @Override
   public void execute(String[] args) throws Exception {
-    ServerContext context = new ServerContext(new SiteConfiguration());
-    AccumuloConfiguration config = context.getServerConfFactory().getSystemConfiguration();
-    Authenticator authenticator = AccumuloVFSClassLoader.getClassLoader()
-        .loadClass(config.get(Property.INSTANCE_SECURITY_AUTHENTICATOR))
-        .asSubclass(Authenticator.class).newInstance();
+    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+      AccumuloConfiguration config = context.getServerConfFactory().getSystemConfiguration();
+      Authenticator authenticator = AccumuloVFSClassLoader.getClassLoader()
+          .loadClass(config.get(Property.INSTANCE_SECURITY_AUTHENTICATOR))
+          .asSubclass(Authenticator.class).newInstance();
 
-    List<Set<TokenProperty>> tokenProps = new ArrayList<>();
+      List<Set<TokenProperty>> tokenProps = new ArrayList<>();
 
-    for (Class<? extends AuthenticationToken> tokenType : authenticator.getSupportedTokenTypes()) {
-      tokenProps.add(tokenType.newInstance().getProperties());
-    }
+      for (Class<? extends AuthenticationToken> tokenType : authenticator
+          .getSupportedTokenTypes()) {
+        tokenProps.add(tokenType.newInstance().getProperties());
+      }
 
-    System.out
-        .println("Supported token types for " + authenticator.getClass().getName() + " are : ");
-    for (Class<? extends AuthenticationToken> tokenType : authenticator.getSupportedTokenTypes()) {
       System.out
-          .println("\t" + tokenType.getName() + ", which accepts the following properties : ");
+          .println("Supported token types for " + authenticator.getClass().getName() + " are : ");
+      for (Class<? extends AuthenticationToken> tokenType : authenticator
+          .getSupportedTokenTypes()) {
+        System.out
+            .println("\t" + tokenType.getName() + ", which accepts the following properties : ");
 
-      for (TokenProperty tokenProperty : tokenType.newInstance().getProperties()) {
-        System.out.println("\t\t" + tokenProperty);
-      }
+        for (TokenProperty tokenProperty : tokenType.newInstance().getProperties()) {
+          System.out.println("\t\t" + tokenProperty);
+        }
 
-      System.out.println();
+        System.out.println();
+      }
     }
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
index eb8979cc31..52e8d6cc46 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/TabletServerLocks.java
@@ -41,34 +41,35 @@
 
   public static void main(String[] args) throws Exception {
 
-    ServerContext context = new ServerContext(new SiteConfiguration());
-    String tserverPath = context.getZooKeeperRoot() + Constants.ZTSERVERS;
-    Opts opts = new Opts();
-    opts.parseArgs(TabletServerLocks.class.getName(), args);
+    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+      String tserverPath = context.getZooKeeperRoot() + Constants.ZTSERVERS;
+      Opts opts = new Opts();
+      opts.parseArgs(TabletServerLocks.class.getName(), args);
 
-    ZooCache cache = context.getZooCache();
-    ZooReaderWriter zoo = context.getZooReaderWriter();
+      ZooCache cache = context.getZooCache();
+      ZooReaderWriter zoo = context.getZooReaderWriter();
 
-    if (opts.list) {
+      if (opts.list) {
 
-      List<String> tabletServers = zoo.getChildren(tserverPath);
+        List<String> tabletServers = zoo.getChildren(tserverPath);
 
-      for (String tabletServer : tabletServers) {
-        byte[] lockData = ZooLock.getLockData(cache, tserverPath + "/" + tabletServer, null);
-        String holder = null;
-        if (lockData != null) {
-          holder = new String(lockData, UTF_8);
-        }
+        for (String tabletServer : tabletServers) {
+          byte[] lockData = ZooLock.getLockData(cache, tserverPath + "/" + tabletServer, null);
+          String holder = null;
+          if (lockData != null) {
+            holder = new String(lockData, UTF_8);
+          }
 
-        System.out.printf("%32s %16s%n", tabletServer, holder);
+          System.out.printf("%32s %16s%n", tabletServer, holder);
+        }
+      } else if (opts.delete != null) {
+        ZooLock.deleteLock(zoo, tserverPath + "/" + args[1]);
+      } else {
+        System.out.println(
+            "Usage : " + TabletServerLocks.class.getName() + " -list|-delete <tserver lock>");
       }
-    } else if (opts.delete != null) {
-      ZooLock.deleteLock(zoo, tserverPath + "/" + args[1]);
-    } else {
-      System.out.println(
-          "Usage : " + TabletServerLocks.class.getName() + " -list|-delete <tserver lock>");
-    }
 
+    }
   }
 
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java b/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
index edcc619425..692b0064e3 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ZooKeeperMain.java
@@ -64,18 +64,19 @@ public String description() {
   public void execute(final String[] args) throws Exception {
     Opts opts = new Opts();
     opts.parseArgs(ZooKeeperMain.class.getName(), args);
-    ServerContext context = new ServerContext(new SiteConfiguration());
-    FileSystem fs = context.getVolumeManager().getDefaultVolume().getFileSystem();
-    String baseDir = ServerConstants.getBaseUris(context.getConfiguration())[0];
-    System.out.println("Using " + fs.makeQualified(new Path(baseDir + "/instance_id"))
-        + " to lookup accumulo instance");
-    if (opts.servers == null) {
-      opts.servers = context.getZooKeepers();
+    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+      FileSystem fs = context.getVolumeManager().getDefaultVolume().getFileSystem();
+      String baseDir = ServerConstants.getBaseUris(context.getConfiguration())[0];
+      System.out.println("Using " + fs.makeQualified(new Path(baseDir + "/instance_id"))
+          + " to lookup accumulo instance");
+      if (opts.servers == null) {
+        opts.servers = context.getZooKeepers();
+      }
+      System.out.println("The accumulo instance id is " + context.getInstanceID());
+      if (!opts.servers.contains("/"))
+        opts.servers += "/accumulo/" + context.getInstanceID();
+      org.apache.zookeeper.ZooKeeperMain
+          .main(new String[] {"-server", opts.servers, "-timeout", "" + (opts.timeout * 1000)});
     }
-    System.out.println("The accumulo instance id is " + context.getInstanceID());
-    if (!opts.servers.contains("/"))
-      opts.servers += "/accumulo/" + context.getInstanceID();
-    org.apache.zookeeper.ZooKeeperMain
-        .main(new String[] {"-server", opts.servers, "-timeout", "" + (opts.timeout * 1000)});
   }
 }
diff --git a/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
index f2cdd3c228..afa736bff7 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/util/FateAdmin.java
@@ -77,28 +77,29 @@ public static void main(String[] args) throws Exception {
 
     AdminUtil<Master> admin = new AdminUtil<>();
 
-    ServerContext context = new ServerContext(new SiteConfiguration());
-    final String zkRoot = context.getZooKeeperRoot();
-    String path = zkRoot + Constants.ZFATE;
-    String masterPath = zkRoot + Constants.ZMASTER_LOCK;
-    IZooReaderWriter zk = context.getZooReaderWriter();
-    ZooStore<Master> zs = new ZooStore<>(path, zk);
+    try (ServerContext context = new ServerContext(new SiteConfiguration())) {
+      final String zkRoot = context.getZooKeeperRoot();
+      String path = zkRoot + Constants.ZFATE;
+      String masterPath = zkRoot + Constants.ZMASTER_LOCK;
+      IZooReaderWriter zk = context.getZooReaderWriter();
+      ZooStore<Master> zs = new ZooStore<>(path, zk);
 
-    if (jc.getParsedCommand().equals("fail")) {
-      for (String txid : txOpts.get(jc.getParsedCommand()).txids) {
-        if (!admin.prepFail(zs, zk, masterPath, txid)) {
-          System.exit(1);
+      if (jc.getParsedCommand().equals("fail")) {
+        for (String txid : txOpts.get(jc.getParsedCommand()).txids) {
+          if (!admin.prepFail(zs, zk, masterPath, txid)) {
+            System.exit(1);
+          }
         }
-      }
-    } else if (jc.getParsedCommand().equals("delete")) {
-      for (String txid : txOpts.get(jc.getParsedCommand()).txids) {
-        if (!admin.prepDelete(zs, zk, masterPath, txid)) {
-          System.exit(1);
+      } else if (jc.getParsedCommand().equals("delete")) {
+        for (String txid : txOpts.get(jc.getParsedCommand()).txids) {
+          if (!admin.prepDelete(zs, zk, masterPath, txid)) {
+            System.exit(1);
+          }
+          admin.deleteLocks(zs, zk, zkRoot + Constants.ZTABLE_LOCKS, txid);
         }
-        admin.deleteLocks(zs, zk, zkRoot + Constants.ZTABLE_LOCKS, txid);
+      } else if (jc.getParsedCommand().equals("print")) {
+        admin.print(new ReadOnlyStore<>(zs), zk, zkRoot + Constants.ZTABLE_LOCKS);
       }
-    } else if (jc.getParsedCommand().equals("print")) {
-      admin.print(new ReadOnlyStore<>(zs), zk, zkRoot + Constants.ZTABLE_LOCKS);
     }
   }
 }
diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
index dd15da6343..99d3e4e8aa 100644
--- a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
+++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java
@@ -304,12 +304,8 @@ private AccumuloClient ensureTraceTableExists(final AccumuloConfiguration conf)
   }
 
   public void run() throws Exception {
-    SimpleTimer.getInstance(serverConfiguration.getSystemConfiguration()).schedule(new Runnable() {
-      @Override
-      public void run() {
-        flush();
-      }
-    }, SCHEDULE_DELAY, SCHEDULE_PERIOD);
+    SimpleTimer.getInstance(serverConfiguration.getSystemConfiguration()).schedule(() -> flush(),
+        SCHEDULE_DELAY, SCHEDULE_PERIOD);
     server.serve();
   }
 
@@ -415,8 +411,7 @@ public static void main(String[] args) throws Exception {
     loginTracer(context.getConfiguration());
     MetricsSystemHelper.configure(TraceServer.class.getSimpleName());
     ServerUtil.init(context, app);
-    TraceServer server = new TraceServer(context, opts.getAddress());
-    try {
+    try (TraceServer server = new TraceServer(context, opts.getAddress())) {
       server.run();
     } finally {
       log.info("tracer stopping");
diff --git a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayerTest.java b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayerTest.java
index 5cc6c8f7c3..0f37beee28 100644
--- a/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayerTest.java
+++ b/server/tserver/src/test/java/org/apache/accumulo/tserver/replication/BatchWriterReplicationReplayerTest.java
@@ -26,7 +26,6 @@
 import java.io.DataOutputStream;
 import java.nio.ByteBuffer;
 
-import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.BatchWriter;
 import org.apache.accumulo.core.client.BatchWriterConfig;
 import org.apache.accumulo.core.clientImpl.ClientContext;
@@ -44,13 +43,9 @@
 
 import com.google.common.collect.Lists;
 
-/**
- *
- */
 public class BatchWriterReplicationReplayerTest {
 
   private ClientContext context;
-  private AccumuloClient client;
   private AccumuloConfiguration conf;
   private BatchWriter bw;
 
diff --git a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
index 855b1741bc..ed4234468b 100644
--- a/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/functional/ReadWriteIT.java
@@ -128,8 +128,9 @@ protected int defaultTimeoutSeconds() {
 
   @Test(expected = RuntimeException.class)
   public void invalidInstanceName() throws Exception {
-    try (AccumuloClient client = Accumulo.newClient().to("fake_instance_name", cluster.getZooKeepers())
-        .as(getAdminPrincipal(), getAdminToken()).build()) {
+    try (AccumuloClient client = Accumulo.newClient()
+        .to("fake_instance_name", cluster.getZooKeepers()).as(getAdminPrincipal(), getAdminToken())
+        .build()) {
       client.instanceOperations().getTabletServers();
     }
   }
diff --git a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
index 7d634b6166..222d7575dc 100644
--- a/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
+++ b/test/src/main/java/org/apache/accumulo/test/server/security/SystemCredentialsIT.java
@@ -59,38 +59,39 @@ public void testSystemCredentials() throws Exception {
 
   public static void main(final String[] args) throws AccumuloException, TableNotFoundException {
     SiteConfiguration siteConfig = new SiteConfiguration();
-    ServerContext context = new ServerContext(siteConfig);
-    Credentials creds;
-    String badInstanceID = SystemCredentials.class.getName();
-    if (args.length < 2)
-      throw new RuntimeException("Incorrect usage; expected to be run by test only");
-    switch (args[0]) {
-      case "bad":
-        creds = SystemCredentials.get(badInstanceID, siteConfig);
-        break;
-      case "good":
-        creds = SystemCredentials.get(context.getInstanceID(), siteConfig);
-        break;
-      case "bad_password":
-        creds = new SystemCredentials(badInstanceID, "!SYSTEM", new PasswordToken("fake"));
-        break;
-      default:
+    try (ServerContext context = new ServerContext(siteConfig)) {
+      Credentials creds;
+      String badInstanceID = SystemCredentials.class.getName();
+      if (args.length < 2)
         throw new RuntimeException("Incorrect usage; expected to be run by test only");
-    }
-    try (AccumuloClient client = Accumulo.newClient().from(context.getProperties())
-        .as(creds.getPrincipal(), creds.getToken()).build()) {
-      client.securityOperations().authenticateUser(creds.getPrincipal(), creds.getToken());
-      try (Scanner scan = client.createScanner(RootTable.NAME, Authorizations.EMPTY)) {
-        for (Entry<Key,Value> e : scan) {
-          e.hashCode();
+      switch (args[0]) {
+        case "bad":
+          creds = SystemCredentials.get(badInstanceID, siteConfig);
+          break;
+        case "good":
+          creds = SystemCredentials.get(context.getInstanceID(), siteConfig);
+          break;
+        case "bad_password":
+          creds = new SystemCredentials(badInstanceID, "!SYSTEM", new PasswordToken("fake"));
+          break;
+        default:
+          throw new RuntimeException("Incorrect usage; expected to be run by test only");
+      }
+      try (AccumuloClient client = Accumulo.newClient().from(context.getProperties())
+          .as(creds.getPrincipal(), creds.getToken()).build()) {
+        client.securityOperations().authenticateUser(creds.getPrincipal(), creds.getToken());
+        try (Scanner scan = client.createScanner(RootTable.NAME, Authorizations.EMPTY)) {
+          for (Entry<Key,Value> e : scan) {
+            e.hashCode();
+          }
+        } catch (RuntimeException e) {
+          e.printStackTrace(System.err);
+          System.exit(SCAN_FAILED);
         }
-      } catch (RuntimeException e) {
+      } catch (AccumuloSecurityException e) {
         e.printStackTrace(System.err);
-        System.exit(SCAN_FAILED);
+        System.exit(AUTHENICATION_FAILED);
       }
-    } catch (AccumuloSecurityException e) {
-      e.printStackTrace(System.err);
-      System.exit(AUTHENICATION_FAILED);
     }
   }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services