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/07 15:23:32 UTC

[GitHub] ctubbsii closed pull request #805: Remove unnecessary exceptions from ClientContext

ctubbsii closed pull request #805: Remove unnecessary exceptions from ClientContext
URL: https://github.com/apache/accumulo/pull/805
 
 
   

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 e1546f4f71..92d6be96ab 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
@@ -482,12 +482,7 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException {
       log.debug("Initializing input split: " + baseSplit);
 
       ClientContext context = new ClientContext(getClientInfo(job));
-      AccumuloClient client;
-      try {
-        client = context.getClient();
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IllegalStateException(e);
-      }
+      AccumuloClient client = context.getClient();
       Authorizations authorizations = getScanAuthorizations(job);
       String classLoaderContext = getClassLoaderContext(job);
       String table = baseSplit.getTableName();
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 235ac32729..b159291f41 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
@@ -490,12 +490,7 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
 
       ClientInfo info = getClientInfo(attempt);
       ClientContext context = new ClientContext(info);
-      AccumuloClient client;
-      try {
-        client = context.getClient();
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IllegalStateException(e);
-      }
+      AccumuloClient client = context.getClient();
       Authorizations authorizations = getScanAuthorizations(attempt);
       String classLoaderContext = getClassLoaderContext(attempt);
       String table = split.getTableName();
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloClientImpl.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloClientImpl.java
index 8657119980..54f111d1be 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloClientImpl.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/AccumuloClientImpl.java
@@ -293,9 +293,9 @@ public void close() {
 
     private Properties properties = new Properties();
     private AuthenticationToken token = null;
-    private Function<ClientBuilderImpl,T> builderFunction;
+    private Function<ClientBuilderImpl<T>,T> builderFunction;
 
-    public ClientBuilderImpl(Function<ClientBuilderImpl,T> builderFunction) {
+    public ClientBuilderImpl(Function<ClientBuilderImpl<T>,T> builderFunction) {
       this.builderFunction = builderFunction;
     }
 
diff --git a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
index d80ddbaa14..16915a32bd 100644
--- a/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
+++ b/core/src/main/java/org/apache/accumulo/core/clientImpl/ClientContext.java
@@ -243,8 +243,7 @@ public SaslConnectionParams getSaslParams() {
   /**
    * Retrieve an Accumulo client
    */
-  public synchronized AccumuloClient getClient()
-      throws AccumuloException, AccumuloSecurityException {
+  public synchronized AccumuloClient getClient() {
     ensureOpen();
     if (client == null) {
       client = new AccumuloClientImpl(SingletonReservation.noop(), this);
diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
index eec663cff6..6434de0fe3 100644
--- a/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
+++ b/core/src/main/java/org/apache/accumulo/core/metadata/schema/MetadataScanner.java
@@ -256,11 +256,7 @@ public MetadataScanner build()
 
     @Override
     public TableOptions from(ClientContext ctx) {
-      try {
-        this.client = ctx.getClient();
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new RuntimeException(e);
-      }
+      this.client = ctx.getClient();
       return this;
     }
 
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
index 7b91397676..c86c310971 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapred/AbstractInputFormat.java
@@ -283,12 +283,7 @@ public void initialize(InputSplit inSplit, JobConf job) throws IOException {
       log.debug("Initializing input split: " + baseSplit);
 
       ClientContext context = new ClientContext(getClientInfo(job));
-      AccumuloClient client;
-      try {
-        client = context.getClient();
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IllegalStateException(e);
-      }
+      AccumuloClient client = context.getClient();
       Authorizations authorizations = getScanAuthorizations(job);
       String classLoaderContext = getClassLoaderContext(job);
       String table = baseSplit.getTableName();
diff --git a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
index 7a9bfb575c..891a354f50 100644
--- a/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
+++ b/hadoop-mapreduce/src/main/java/org/apache/accumulo/hadoopImpl/mapreduce/AbstractInputFormat.java
@@ -287,12 +287,7 @@ public void initialize(InputSplit inSplit, TaskAttemptContext attempt) throws IO
 
       ClientInfo info = getClientInfo(attempt);
       ClientContext context = new ClientContext(info);
-      AccumuloClient client;
-      try {
-        client = context.getClient();
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new IllegalStateException(e);
-      }
+      AccumuloClient client = context.getClient();
       Authorizations authorizations = getScanAuthorizations(attempt);
       String classLoaderContext = getClassLoaderContext(attempt);
       String table = split.getTableName();
diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
index 8106baf10b..bf5240edce 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/ServerContext.java
@@ -25,8 +25,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.client.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.ClientInfo;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.clientImpl.AccumuloClientImpl;
@@ -242,8 +240,7 @@ public AuthenticationTokenSecretManager getSecretManager() {
   }
 
   @Override
-  public synchronized AccumuloClient getClient()
-      throws AccumuloException, AccumuloSecurityException {
+  public synchronized AccumuloClient getClient() {
     if (client == null) {
       client = new AccumuloClientImpl(SingletonReservation.noop(), this);
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
index 268ffafe72..2aa7cbeddc 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/client/ClientServiceHandler.java
@@ -28,7 +28,6 @@
 import java.util.TreeSet;
 
 import org.apache.accumulo.core.Constants;
-import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.NamespaceNotFoundException;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -476,9 +475,7 @@ public boolean checkNamespaceClass(TInfo tinfo, TCredentials credentials, String
       }
       return retUsages;
 
-    } catch (AccumuloSecurityException e) {
-      throw e.asThriftException();
-    } catch (AccumuloException | TableNotFoundException | IOException e) {
+    } catch (TableNotFoundException | IOException e) {
       throw new TException(e);
     }
   }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
index b763f641c7..8fadccdfa2 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/master/balancer/TableLoadBalancer.java
@@ -25,8 +25,6 @@
 import java.util.Set;
 import java.util.SortedMap;
 
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.admin.TableOperations;
 import org.apache.accumulo.core.clientImpl.Table;
 import org.apache.accumulo.core.conf.Property;
@@ -138,11 +136,7 @@ public void getAssignments(SortedMap<TServerInstance,TabletServerStatus> current
 
   protected TableOperations getTableOperations() {
     if (tops == null)
-      try {
-        tops = this.context.getClient().tableOperations();
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        log.error("Unable to access table operations from within table balancer", e);
-      }
+      tops = this.context.getClient().tableOperations();
     return tops;
   }
 
diff --git a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
index c1ab8859f4..a58a4f1018 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/replication/ReplicationUtil.java
@@ -27,8 +27,6 @@
 import java.util.Set;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.BatchScanner;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
@@ -164,7 +162,7 @@ public int getMaxReplicationThreads(MasterMonitorInfo mmi) {
     BatchScanner bs;
     try {
       bs = context.getClient().createBatchScanner(ReplicationTable.NAME, Authorizations.EMPTY, 4);
-    } catch (TableNotFoundException | AccumuloException | AccumuloSecurityException e) {
+    } catch (TableNotFoundException e) {
       log.debug("No replication table exists", e);
       return counts;
     }
@@ -201,7 +199,7 @@ public int getMaxReplicationThreads(MasterMonitorInfo mmi) {
     BatchScanner bs;
     try {
       bs = context.getClient().createBatchScanner(ReplicationTable.NAME, Authorizations.EMPTY, 4);
-    } catch (TableNotFoundException | AccumuloException | AccumuloSecurityException e) {
+    } catch (TableNotFoundException e) {
       log.debug("No replication table exists", e);
       return paths;
     }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
index 1aa90edacb..5ec0000965 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/util/ReplicationTableUtil.java
@@ -84,12 +84,7 @@ static synchronized void addWriter(Credentials creds, Writer writer) {
   static synchronized Writer getWriter(ClientContext context) {
     Writer replicationTable = writers.get(context.getCredentials());
     if (replicationTable == null) {
-      AccumuloClient client;
-      try {
-        client = context.getClient();
-      } catch (AccumuloException | AccumuloSecurityException e) {
-        throw new RuntimeException(e);
-      }
+      AccumuloClient client = context.getClient();
 
       configureMetadataTable(client, MetadataTable.NAME);
 
diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
index 0458c28081..eae596980c 100644
--- a/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
+++ b/server/gc/src/main/java/org/apache/accumulo/gc/GarbageCollectWriteAheadLogs.java
@@ -29,8 +29,6 @@
 import java.util.UUID;
 
 import org.apache.accumulo.core.client.AccumuloClient;
-import org.apache.accumulo.core.client.AccumuloException;
-import org.apache.accumulo.core.client.AccumuloSecurityException;
 import org.apache.accumulo.core.client.Scanner;
 import org.apache.accumulo.core.client.TableNotFoundException;
 import org.apache.accumulo.core.data.Key;
@@ -53,7 +51,6 @@
 import org.apache.accumulo.server.log.WalStateManager.WalMarkerException;
 import org.apache.accumulo.server.log.WalStateManager.WalState;
 import org.apache.accumulo.server.master.LiveTServerSet;
-import org.apache.accumulo.server.master.LiveTServerSet.Listener;
 import org.apache.accumulo.server.master.state.MetaDataStateStore;
 import org.apache.accumulo.server.master.state.RootTabletStateStore;
 import org.apache.accumulo.server.master.state.TServerInstance;
@@ -94,23 +91,14 @@
     this.context = context;
     this.fs = fs;
     this.useTrash = useTrash;
-    this.liveServers = new LiveTServerSet(context, new Listener() {
-      @Override
-      public void update(LiveTServerSet current, Set<TServerInstance> deleted,
-          Set<TServerInstance> added) {
-        log.debug("New tablet servers noticed: {}", added);
-        log.debug("Tablet servers removed: {}", deleted);
-      }
+    this.liveServers = new LiveTServerSet(context, (current, deleted, added) -> {
+      log.debug("New tablet servers noticed: {}", added);
+      log.debug("Tablet servers removed: {}", deleted);
     });
     liveServers.startListeningForTabletServerChanges();
     this.walMarker = new WalStateManager(context);
-    this.store = new Iterable<TabletLocationState>() {
-      @Override
-      public Iterator<TabletLocationState> iterator() {
-        return Iterators.concat(new RootTabletStateStore(context).iterator(),
-            new MetaDataStateStore(context).iterator());
-      }
-    };
+    this.store = () -> Iterators.concat(new RootTabletStateStore(context).iterator(),
+        new MetaDataStateStore(context).iterator());
   }
 
   /**
@@ -377,7 +365,7 @@ protected int removeReplicationEntries(Map<UUID,TServerInstance> candidates)
       }
 
       return candidates.size();
-    } catch (AccumuloException | AccumuloSecurityException | TableNotFoundException e) {
+    } catch (TableNotFoundException e) {
       log.error("Failed to scan metadata table", e);
       throw new IllegalArgumentException(e);
     }


 

----------------------------------------------------------------
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