You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lens.apache.org by am...@apache.org on 2015/04/15 21:47:26 UTC

[02/30] incubator-lens git commit: LENS-273 : Changes locking for persisting server state and makes buffer size of persist out to be configurable (Himanshu Gahlaut via amareshwari)

LENS-273 : Changes locking for persisting server state and makes buffer size of persist out to be configurable (Himanshu Gahlaut via amareshwari)

Conflicts:
	lens-server/src/main/java/org/apache/lens/server/LensServices.java


Project: http://git-wip-us.apache.org/repos/asf/incubator-lens/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-lens/commit/77cea318
Tree: http://git-wip-us.apache.org/repos/asf/incubator-lens/tree/77cea318
Diff: http://git-wip-us.apache.org/repos/asf/incubator-lens/diff/77cea318

Branch: refs/heads/master
Commit: 77cea318b21b771d972f9ea61ecaab309cd48ec2
Parents: b82db5a
Author: Amareshwari Sriramdasu <am...@inmobi.com>
Authored: Fri Feb 6 17:53:36 2015 +0530
Committer: Amareshwari Sriramdasu <am...@inmobi.com>
Committed: Fri Feb 6 17:59:21 2015 +0530

----------------------------------------------------------------------
 .../lens/server/api/LensConfConstants.java      | 10 +++
 .../org/apache/lens/server/LensServices.java    | 88 ++++++++++++--------
 .../src/main/resources/lensserver-default.xml   |  6 ++
 src/site/apt/admin/config.apt                   | 56 +++++++------
 4 files changed, 96 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/77cea318/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java b/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
index 0ae539e..b4cef60 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/LensConfConstants.java
@@ -763,4 +763,14 @@ public final class LensConfConstants {
    * The Constant SERVICE_PROVIDER_FACTORY.
    */
   public static final String SERVICE_PROVIDER_FACTORY = SERVER_PFX + "service.provider.factory";
+
+  /**
+   * Key for reading Output Stream Buffer Size used in writing lens server state to file system
+   */
+  public static final String STATE_PERSIST_OUT_STREAM_BUFF_SIZE = SERVER_PFX + "state.persist.out.stream.buffer.size";
+
+  /**
+   * Default Output Stream Buffer Size used in writing lens server state to file system: 1MB
+   */
+  public static final int DEFAULT_STATE_PERSIST_OUT_STREAM_BUFF_SIZE = 1048576;
 }

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/77cea318/lens-server/src/main/java/org/apache/lens/server/LensServices.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/LensServices.java b/lens-server/src/main/java/org/apache/lens/server/LensServices.java
index 96158dd..e30335c 100644
--- a/lens-server/src/main/java/org/apache/lens/server/LensServices.java
+++ b/lens-server/src/main/java/org/apache/lens/server/LensServices.java
@@ -18,6 +18,8 @@
  */
 package org.apache.lens.server;
 
+import static org.apache.lens.server.api.LensConfConstants.*;
+
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.ObjectInputStream;
@@ -28,6 +30,13 @@ import java.util.*;
 import lombok.Getter;
 import lombok.Setter;
 
+import org.apache.lens.server.api.ServiceProvider;
+import org.apache.lens.server.api.events.LensEventService;
+import org.apache.lens.server.api.metrics.MetricsService;
+import org.apache.lens.server.session.LensSessionImpl;
+import org.apache.lens.server.stats.StatisticsService;
+import org.apache.lens.server.user.UserConfigLoaderFactory;
+
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -38,13 +47,6 @@ import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.Service;
 import org.apache.hive.service.cli.CLIService;
-import org.apache.lens.server.api.LensConfConstants;
-import org.apache.lens.server.api.ServiceProvider;
-import org.apache.lens.server.api.events.LensEventService;
-import org.apache.lens.server.api.metrics.MetricsService;
-import org.apache.lens.server.session.LensSessionImpl;
-import org.apache.lens.server.stats.StatisticsService;
-import org.apache.lens.server.user.UserConfigLoaderFactory;
 
 /**
  * Manage lifecycle of all Lens services
@@ -59,6 +61,7 @@ public class LensServices extends CompositeService implements ServiceProvider {
 
   /** Constant for FileSystem auto close on shutdown config */
   private static final String FS_AUTOMATIC_CLOSE = "fs.automatic.close";
+  private static final String FS_IO_FILE_BUFFER_SIZE = "io.file.buffer.size";
 
   /** The instance. */
   private static LensServices INSTANCE = new LensServices(LENS_SERVICES_NAME);
@@ -95,6 +98,9 @@ public class LensServices extends CompositeService implements ServiceProvider {
   /** The timer. */
   private Timer timer;
 
+  /* Lock for synchronizing persistence of LensServices state */
+  private final Object statePersistenceLock = new Object();
+
   /**
    * The Enum SERVICE_MODE.
    */
@@ -139,8 +145,8 @@ public class LensServices extends CompositeService implements ServiceProvider {
       conf.addResource("lensserver-default.xml");
       conf.addResource("lens-site.xml");
       conf.setVar(HiveConf.ConfVars.HIVE_SESSION_IMPL_CLASSNAME, LensSessionImpl.class.getCanonicalName());
-      serviceMode = conf.getEnum(LensConfConstants.SERVER_MODE,
-          SERVICE_MODE.valueOf(LensConfConstants.DEFAULT_SERVER_MODE));
+      serviceMode = conf.getEnum(SERVER_MODE,
+        SERVICE_MODE.valueOf(DEFAULT_SERVER_MODE));
       cliService = new CLIService();
 
       // Add default services
@@ -151,11 +157,11 @@ public class LensServices extends CompositeService implements ServiceProvider {
 
       // Add configured services, these are instances of LensService which need a CLIService instance
       // for session management
-      String[] serviceNames = conf.getStrings(LensConfConstants.SERVICE_NAMES);
+      String[] serviceNames = conf.getStrings(SERVICE_NAMES);
       for (String sName : serviceNames) {
         try {
 
-          String serviceClassName = conf.get(LensConfConstants.getServiceImplConfKey(sName));
+          String serviceClassName = conf.get(getServiceImplConfKey(sName));
 
           if (StringUtils.isBlank(serviceClassName)) {
             LOG.warn("Invalid class for service " + sName + " class=" + serviceClassName);
@@ -193,20 +199,25 @@ public class LensServices extends CompositeService implements ServiceProvider {
       super.init(conf);
 
       // setup persisted state
-      String persistPathStr = conf.get(LensConfConstants.SERVER_STATE_PERSIST_LOCATION,
-          LensConfConstants.DEFAULT_SERVER_STATE_PERSIST_LOCATION);
+      String persistPathStr = conf.get(SERVER_STATE_PERSIST_LOCATION,
+        DEFAULT_SERVER_STATE_PERSIST_LOCATION);
       persistDir = new Path(persistPathStr);
       try {
         Configuration configuration = new Configuration(conf);
         configuration.setBoolean(FS_AUTOMATIC_CLOSE, false);
+
+        int outStreamBufferSize = conf.getInt(STATE_PERSIST_OUT_STREAM_BUFF_SIZE,
+            DEFAULT_STATE_PERSIST_OUT_STREAM_BUFF_SIZE);
+        configuration.setInt(FS_IO_FILE_BUFFER_SIZE, outStreamBufferSize);
+        LOG.info("STATE_PERSIST_OUT_STREAM_BUFF_SIZE IN BYTES:"+outStreamBufferSize);
         persistenceFS = FileSystem.newInstance(persistDir.toUri(), configuration);
         setupPersistedState();
       } catch (Exception e) {
         LOG.error("Could not recover from persisted state", e);
         throw new RuntimeException("Could not recover from persisted state", e);
       }
-      snapShotInterval = conf.getLong(LensConfConstants.SERVER_SNAPSHOT_INTERVAL,
-          LensConfConstants.DEFAULT_SERVER_SNAPSHOT_INTERVAL);
+      snapShotInterval = conf.getLong(SERVER_SNAPSHOT_INTERVAL,
+        DEFAULT_SERVER_SNAPSHOT_INTERVAL);
       LOG.info("Initialized services: " + services.keySet().toString());
       UserConfigLoaderFactory.init(conf);
       timer = new Timer("lens-server-snapshotter", true);
@@ -244,8 +255,8 @@ public class LensServices extends CompositeService implements ServiceProvider {
    *           the class not found exception
    */
   private void setupPersistedState() throws IOException, ClassNotFoundException {
-    if (conf.getBoolean(LensConfConstants.SERVER_RECOVER_ON_RESTART,
-        LensConfConstants.DEFAULT_SERVER_RECOVER_ON_RESTART)) {
+    if (conf.getBoolean(SERVER_RECOVER_ON_RESTART,
+      DEFAULT_SERVER_RECOVER_ON_RESTART)) {
 
       for (LensService service : lensServices) {
         ObjectInputStream in = null;
@@ -273,30 +284,33 @@ public class LensServices extends CompositeService implements ServiceProvider {
    * @throws IOException
    *           Signals that an I/O exception has occurred.
    */
-  private synchronized void persistLensServiceState() throws IOException {
-    if (conf.getBoolean(LensConfConstants.SERVER_RESTART_ENABLED, LensConfConstants.DEFAULT_SERVER_RESTART_ENABLED)) {
-      if (persistDir != null) {
-        LOG.info("Persisting server state in " + persistDir);
-
-        for (LensService service : lensServices) {
-          LOG.info("Persisting state of service:" + service.getName());
-          Path serviceWritePath = new Path(persistDir, service.getName() + ".out");
-          ObjectOutputStream out = null;
-          try {
-            out = new ObjectOutputStream(persistenceFS.create(serviceWritePath));
-            service.writeExternal(out);
-          } finally {
-            if (out != null) {
-              out.close();
+  private void persistLensServiceState() throws IOException {
+
+    synchronized (statePersistenceLock) {
+      if (conf.getBoolean(SERVER_RESTART_ENABLED, DEFAULT_SERVER_RESTART_ENABLED)) {
+        if (persistDir != null) {
+          LOG.info("Persisting server state in " + persistDir);
+
+          for (LensService service : lensServices) {
+            LOG.info("Persisting state of service:" + service.getName());
+            Path serviceWritePath = new Path(persistDir, service.getName() + ".out");
+            ObjectOutputStream out = null;
+            try {
+              out = new ObjectOutputStream(persistenceFS.create(serviceWritePath));
+              service.writeExternal(out);
+            } finally {
+              if (out != null) {
+                out.close();
+              }
             }
+            Path servicePath = getServicePersistPath(service);
+            persistenceFS.rename(serviceWritePath, servicePath);
+            LOG.info("Persisted service " + service.getName() + " to " + servicePath);
           }
-          Path servicePath = getServicePersistPath(service);
-          persistenceFS.rename(serviceWritePath, servicePath);
-          LOG.info("Persisted service " + service.getName() + " to " + servicePath);
         }
+      } else {
+        LOG.info("Server restart is not enabled. Not persisting the server state");
       }
-    } else {
-      LOG.info("Server restart is not enabled. Not persisting the server state");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/77cea318/lens-server/src/main/resources/lensserver-default.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/main/resources/lensserver-default.xml b/lens-server/src/main/resources/lensserver-default.xml
index e983c7e..e9a0a94 100644
--- a/lens-server/src/main/resources/lensserver-default.xml
+++ b/lens-server/src/main/resources/lensserver-default.xml
@@ -556,4 +556,10 @@
     This is not supposed to be overridden by users.</description>
 </property>
 
+  <property>
+    <name>lens.server.state.persist.out.stream.buffer.size</name>
+    <value>1048576</value>
+    <description>Output Stream Buffer Size used in writing lens server state to file system. Size is in bytes.</description>
+  </property>
+
 </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-lens/blob/77cea318/src/site/apt/admin/config.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/config.apt b/src/site/apt/admin/config.apt
index 7c41e08..c5f3b2f 100644
--- a/src/site/apt/admin/config.apt
+++ b/src/site/apt/admin/config.apt
@@ -114,7 +114,7 @@ Lens server configuration
 *--+--+---+--+
 |45|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
 *--+--+---+--+
-|46|lens.server.service.provider.factory|org.apache.lens.server.ServiceProviderFactoryImpl|Service provider factory implementation class. This parameter is used to lookup the factory implementation class name that would provide an instance of ServiceProvider. Users should instantiate the class to obtain its instance. Example -- Class spfClass 				= conf.getClass("lens.server.service.provider.factory", 	null, ServiceProviderFactory.class); ServiceProviderFactory spf = spfClass.newInstance(); 		ServiceProvider serviceProvider = spf.getServiceProvider(); 		-- This is not supposed to be overridden by users.|
+|46|lens.server.service.provider.factory|org.apache.lens.server.ServiceProviderFactoryImpl|Service provider factory implementation class. This parameter is used to lookup the factory implementation class name that would provide an instance of ServiceProvider. Users should instantiate the class to obtain its instance. Example -- Class spfClass = conf.getClass("lens.server.service.provider.factory", null, ServiceProviderFactory.class); ServiceProviderFactory spf = spfClass.newInstance(); ServiceProvider serviceProvider = spf.getServiceProvider(); -- This is not supposed to be overridden by users.|
 *--+--+---+--+
 |47|lens.server.servicenames|session,query,metastore,scheduler,quota|These services would be started in the specified order when lens-server starts up|
 *--+--+---+--+
@@ -126,56 +126,58 @@ Lens server configuration
 *--+--+---+--+
 |51|lens.server.snapshot.interval|300000|Snapshot interval time in miliseconds for saving lens server state.|
 *--+--+---+--+
-|52|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
+|52|lens.server.state.persist.out.stream.buffer.size|1048576|Output Stream Buffer Size used in writing lens server state to file system. Size is in bytes.|
 *--+--+---+--+
-|53|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
+|53|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
 *--+--+---+--+
-|54|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
+|54|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
 *--+--+---+--+
-|55|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
+|55|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
 *--+--+---+--+
-|56|lens.server.ui.base.uri|http://0.0.0.0:19999/|The base url for the Lens UI Server|
+|56|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
 *--+--+---+--+
-|57|lens.server.ui.enable.caching|true|Set this to false to disable static file caching in the UI server|
+|57|lens.server.ui.base.uri|http://0.0.0.0:19999/|The base url for the Lens UI Server|
 *--+--+---+--+
-|58|lens.server.ui.static.dir|webapp/lens-server/static|The base directory to server UI static files from|
+|58|lens.server.ui.enable.caching|true|Set this to false to disable static file caching in the UI server|
 *--+--+---+--+
-|59|lens.server.user.resolver.custom.class|full.package.name.Classname|Required for CUSTOM user resolver. In case the provided implementations are not sufficient for user config resolver, a custom classname can be provided. Class should extend org.apache.lens.server.user.UserConfigLoader|
+|59|lens.server.ui.static.dir|webapp/lens-server/static|The base directory to server UI static files from|
 *--+--+---+--+
-|60|lens.server.user.resolver.db.keys|lens.session.cluster.user,mapred.job.queue.name|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loaders, the conf keys that will be loaded from database.|
+|60|lens.server.user.resolver.custom.class|full.package.name.Classname|Required for CUSTOM user resolver. In case the provided implementations are not sufficient for user config resolver, a custom classname can be provided. Class should extend org.apache.lens.server.user.UserConfigLoader|
 *--+--+---+--+
-|61|lens.server.user.resolver.db.query|select clusteruser,queue from user_config_table where username=?|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loader, this query will be run with single argument = logged in user and the result columns will be assigned to lens.server.user.resolver.db.keys in order. For ldap backed database resolver, the argument to this query will be the intermediate values obtained from ldap.|
+|61|lens.server.user.resolver.db.keys|lens.session.cluster.user,mapred.job.queue.name|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loaders, the conf keys that will be loaded from database.|
 *--+--+---+--+
-|62|lens.server.user.resolver.fixed.value| |Required for FIXED user resolver. when lens.server.user.resolver.type=FIXED, This will be the value cluster user will resolve to.|
+|62|lens.server.user.resolver.db.query|select clusteruser,queue from user_config_table where username=?|Required for DATABASE and LDAP_BACKED_DATABASE user resolvers. For database based user config loader, this query will be run with single argument = logged in user and the result columns will be assigned to lens.server.user.resolver.db.keys in order. For ldap backed database resolver, the argument to this query will be the intermediate values obtained from ldap.|
 *--+--+---+--+
-|63|lens.server.user.resolver.ldap.bind.dn| |Required for LDAP_BACKED_DATABASE user resolvers. ldap dn for admin binding example: CN=company-it-admin,ou=service-account,ou=company-service-account,dc=dc1,dc=com...|
+|63|lens.server.user.resolver.fixed.value| |Required for FIXED user resolver. when lens.server.user.resolver.type=FIXED, This will be the value cluster user will resolve to.|
 *--+--+---+--+
-|64|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
+|64|lens.server.user.resolver.ldap.bind.dn| |Required for LDAP_BACKED_DATABASE user resolvers. ldap dn for admin binding example: CN=company-it-admin,ou=service-account,ou=company-service-account,dc=dc1,dc=com...|
 *--+--+---+--+
-|65|lens.server.user.resolver.ldap.fields|department|Required for LDAP_BACKED_DATABASE user resolvers. list of fields to be obtained from ldap. These will be cached by the intermediate db.|
+|65|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
 *--+--+---+--+
-|66|lens.server.user.resolver.ldap.intermediate.db.delete.sql|delete from user_department where username=?|Required for LDAP_BACKED_DATABASE user resolvers. query to delete intermediate values from database backing ldap as cache. one argument: logged in user.|
+|66|lens.server.user.resolver.ldap.fields|department|Required for LDAP_BACKED_DATABASE user resolvers. list of fields to be obtained from ldap. These will be cached by the intermediate db.|
 *--+--+---+--+
-|67|lens.server.user.resolver.ldap.intermediate.db.insert.sql|insert into user_department (username, department, expiry) values (?, ?, ?)|Required for LDAP_BACKED_DATABASE user resolvers. query to insert intermediate values from database backing ldap as cache. arguments: first logged in user, then all intermediate values, then current time + expiration time|
+|67|lens.server.user.resolver.ldap.intermediate.db.delete.sql|delete from user_department where username=?|Required for LDAP_BACKED_DATABASE user resolvers. query to delete intermediate values from database backing ldap as cache. one argument: logged in user.|
 *--+--+---+--+
-|68|lens.server.user.resolver.ldap.intermediate.db.query|select department from user_department where username=? and expiry>?|Required for LDAP_BACKED_DATABASE user resolvers. query to obtain intermediate values from database backing ldap as cache. two arguments: logged in user and current time.|
+|68|lens.server.user.resolver.ldap.intermediate.db.insert.sql|insert into user_department (username, department, expiry) values (?, ?, ?)|Required for LDAP_BACKED_DATABASE user resolvers. query to insert intermediate values from database backing ldap as cache. arguments: first logged in user, then all intermediate values, then current time + expiration time|
 *--+--+---+--+
-|69|lens.server.user.resolver.ldap.search.base| |Required for LDAP_BACKED_DATABASE user resolvers. for searching intermediate values for a user, the search keys. example: cn=users,dc=dc1,dc=dc2...|
+|69|lens.server.user.resolver.ldap.intermediate.db.query|select department from user_department where username=? and expiry>?|Required for LDAP_BACKED_DATABASE user resolvers. query to obtain intermediate values from database backing ldap as cache. two arguments: logged in user and current time.|
 *--+--+---+--+
-|70|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
+|70|lens.server.user.resolver.ldap.search.base| |Required for LDAP_BACKED_DATABASE user resolvers. for searching intermediate values for a user, the search keys. example: cn=users,dc=dc1,dc=dc2...|
 *--+--+---+--+
-|71|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
+|71|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
 *--+--+---+--+
-|72|lens.server.user.resolver.propertybased.filename|/path/to/propertyfile|Required for PROPERTYBASED user resolver. when lens.server.user.resolver.type is PROPERTYBASED, then this file will be read and parsed to determine cluster user. Each line should contain username followed by DOT followed by property full name followed by equal-to sign and followed by value. example schema of the file is: user1.lens.server.cluster.user=clusteruser1 user1.mapred.job.queue.name=queue1 *.lens.server.cluster.user=defaultclusteruser *.mapred.job.queue.name=default|
+|72|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
 *--+--+---+--+
-|73|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
+|73|lens.server.user.resolver.propertybased.filename|/path/to/propertyfile|Required for PROPERTYBASED user resolver. when lens.server.user.resolver.type is PROPERTYBASED, then this file will be read and parsed to determine cluster user. Each line should contain username followed by DOT followed by property full name followed by equal-to sign and followed by value. example schema of the file is: user1.lens.server.cluster.user=clusteruser1 user1.mapred.job.queue.name=queue1 *.lens.server.cluster.user=defaultclusteruser *.mapred.job.queue.name=default|
 *--+--+---+--+
-|74|lens.server.ws.featurenames|multipart|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
+|74|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
 *--+--+---+--+
-|75|lens.server.ws.filternames|authentication,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
+|75|lens.server.ws.featurenames|multipart|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|76|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
+|76|lens.server.ws.filternames|authentication,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|77|lens.server.ws.resourcenames|session,metastore,query,quota,scheduler,index|These JAX-RS resources would be started in the specified order when lens-server starts up|
+|77|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
+*--+--+---+--+
+|78|lens.server.ws.resourcenames|session,metastore,query,quota,scheduler,index|These JAX-RS resources would be started in the specified order when lens-server starts up|
 *--+--+---+--+
 The configuration parameters and their default values