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/01/11 22:41:15 UTC

[GitHub] ctubbsii closed pull request #351: ACCUMULO-1975 Refactor Property.createInstanceFromPropertyName into AccumuloConfiguration.instantiateClassProperty

ctubbsii closed pull request #351: ACCUMULO-1975 Refactor Property.createInstanceFromPropertyName into AccumuloConfiguration.instantiateClassProperty
URL: https://github.com/apache/accumulo/pull/351
 
 
   

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/conf/AccumuloConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
index 2c53407edb..832bebdd93 100644
--- a/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
+++ b/core/src/main/java/org/apache/accumulo/core/conf/AccumuloConfiguration.java
@@ -27,7 +27,6 @@
 import org.apache.accumulo.core.Constants;
 import org.apache.accumulo.core.conf.PropertyType.PortRange;
 import org.apache.accumulo.core.util.Pair;
-import org.apache.accumulo.start.classloader.vfs.AccumuloVFSClassLoader;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -284,35 +283,4 @@ public int getMaxFilesPerTablet() {
    */
   public void invalidateCache() {}
 
-  /**
-   * Creates a new instance of a class specified in a configuration property.
-   *
-   * @param property
-   *          property specifying class name
-   * @param base
-   *          base class of type
-   * @param defaultInstance
-   *          instance to use if creation fails
-   * @return new class instance, or default instance if creation failed
-   * @see AccumuloVFSClassLoader
-   */
-  public <T> T instantiateClassProperty(Property property, Class<T> base, T defaultInstance) {
-    String clazzName = get(property);
-    T instance = null;
-
-    try {
-      Class<? extends T> clazz = AccumuloVFSClassLoader.loadClass(clazzName, base);
-      instance = clazz.newInstance();
-      log.info("Loaded class : {}", clazzName);
-    } catch (Exception e) {
-      log.warn("Failed to load class ", e);
-    }
-
-    if (instance == null) {
-      log.info("Using {}", defaultInstance.getClass().getName());
-      instance = defaultInstance;
-    }
-    return instance;
-  }
-
 }
diff --git a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
index 1918646aa0..b859e3f3db 100644
--- a/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
+++ b/server/base/src/main/java/org/apache/accumulo/server/security/SecurityOperation.java
@@ -33,6 +33,7 @@
 import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.SiteConfiguration;
 import org.apache.accumulo.core.data.thrift.IterInfo;
@@ -89,21 +90,23 @@ public static synchronized SecurityOperation getInstance(AccumuloServerContext c
   }
 
   protected static Authorizor getAuthorizor(String instanceId, boolean initialize) {
-    Authorizor toRet = SiteConfiguration.getInstance().instantiateClassProperty(Property.INSTANCE_SECURITY_AUTHORIZOR, Authorizor.class,
-        ZKAuthorizor.getInstance());
+    AccumuloConfiguration conf = SiteConfiguration.getInstance();
+    Authorizor toRet = Property.createInstanceFromPropertyName(conf, Property.INSTANCE_SECURITY_AUTHORIZOR, Authorizor.class, ZKAuthorizor.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
   }
 
   protected static Authenticator getAuthenticator(String instanceId, boolean initialize) {
-    Authenticator toRet = SiteConfiguration.getInstance().instantiateClassProperty(Property.INSTANCE_SECURITY_AUTHENTICATOR, Authenticator.class,
+    AccumuloConfiguration conf = SiteConfiguration.getInstance();
+    Authenticator toRet = Property.createInstanceFromPropertyName(conf, Property.INSTANCE_SECURITY_AUTHENTICATOR, Authenticator.class,
         ZKAuthenticator.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
   }
 
   protected static PermissionHandler getPermHandler(String instanceId, boolean initialize) {
-    PermissionHandler toRet = SiteConfiguration.getInstance().instantiateClassProperty(Property.INSTANCE_SECURITY_PERMISSION_HANDLER, PermissionHandler.class,
+    AccumuloConfiguration conf = SiteConfiguration.getInstance();
+    PermissionHandler toRet = Property.createInstanceFromPropertyName(conf, Property.INSTANCE_SECURITY_PERMISSION_HANDLER, PermissionHandler.class,
         ZKPermHandler.getInstance());
     toRet.initialize(instanceId, initialize);
     return toRet;
diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java
index 4532129d17..441a265eed 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/Master.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java
@@ -610,7 +610,7 @@ public Master(Instance instance, ServerConfigurationFactory config, VolumeManage
     timeKeeper = new MasterTime(this);
     ThriftTransportPool.getInstance().setIdleTime(aconf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT));
     tserverSet = new LiveTServerSet(this, this);
-    this.tabletBalancer = aconf.instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
+    this.tabletBalancer = Property.createInstanceFromPropertyName(aconf, Property.MASTER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
     this.tabletBalancer.init(this);
 
     try {
diff --git a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
index ee7542e06a..273784b600 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/MasterClientServiceHandler.java
@@ -49,6 +49,7 @@
 import org.apache.accumulo.core.client.impl.thrift.TableOperationExceptionType;
 import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException;
 import org.apache.accumulo.core.client.impl.thrift.ThriftTableOperationException;
+import org.apache.accumulo.core.conf.AccumuloConfiguration;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.data.Key;
 import org.apache.accumulo.core.data.Range;
@@ -467,8 +468,8 @@ private void alterTableProperty(TCredentials c, String tableName, String propert
 
   private void updatePlugins(String property) {
     if (property.equals(Property.MASTER_TABLET_BALANCER.getKey())) {
-      TabletBalancer balancer = master.getConfiguration().instantiateClassProperty(Property.MASTER_TABLET_BALANCER, TabletBalancer.class,
-          new DefaultLoadBalancer());
+      AccumuloConfiguration conf = master.getConfiguration();
+      TabletBalancer balancer = Property.createInstanceFromPropertyName(conf, Property.MASTER_TABLET_BALANCER, TabletBalancer.class, new DefaultLoadBalancer());
       balancer.init(master);
       master.tabletBalancer = balancer;
       log.info("tablet balancer changed to {}", master.tabletBalancer.getClass().getName());
diff --git a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
index a6a97c931c..d22d8003c8 100644
--- a/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
+++ b/server/master/src/main/java/org/apache/accumulo/master/recovery/RecoveryManager.java
@@ -173,7 +173,8 @@ public boolean recoverLogs(KeyExtent extent, Collection<Collection<String>> walo
         synchronized (this) {
           if (!closeTasksQueued.contains(sortId) && !sortsQueued.contains(sortId)) {
             AccumuloConfiguration aconf = master.getConfiguration();
-            LogCloser closer = aconf.instantiateClassProperty(Property.MASTER_WALOG_CLOSER_IMPLEMETATION, LogCloser.class, new HadoopLogCloser());
+            LogCloser closer = Property.createInstanceFromPropertyName(aconf, Property.MASTER_WALOG_CLOSER_IMPLEMETATION, LogCloser.class,
+                new HadoopLogCloser());
             Long delay = recoveryDelay.get(sortId);
             if (delay == null) {
               delay = aconf.getTimeInMillis(Property.MASTER_RECOVERY_DELAY);


 

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