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 2016/03/02 13:49:12 UTC

[1/2] lens git commit: LENS-833 : Limit number of open sessions per user on session service

Repository: lens
Updated Branches:
  refs/heads/master 935647ca0 -> 0ba17ef5c


http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/src/site/apt/admin/config.apt
----------------------------------------------------------------------
diff --git a/src/site/apt/admin/config.apt b/src/site/apt/admin/config.apt
index fb15d83..6df9796 100644
--- a/src/site/apt/admin/config.apt
+++ b/src/site/apt/admin/config.apt
@@ -105,152 +105,154 @@ Lens server configuration
 *--+--+---+--+
 |39|lens.server.mail.smtp.timeout|30000|Socket read timeout value in milliseconds. This timeout is implemented by java.net.Socket. Default is 30 seconds.|
 *--+--+---+--+
-|40|lens.server.metastore.service.impl|org.apache.lens.server.metastore.CubeMetastoreServiceImpl|Implementation class for metastore service|
+|40|lens.server.max.sessions.per.user|10|Number of sessions can be allowed for each user. User has to close one of the active sessions to open a new session once limit is reached. Otherwise Server throws an exception by saying that opened session limit has been already reached for user.|
 *--+--+---+--+
-|41|lens.server.metastore.ws.resource.impl|org.apache.lens.server.metastore.MetastoreResource|Implementation class for Metastore Resource|
+|41|lens.server.metastore.service.impl|org.apache.lens.server.metastore.CubeMetastoreServiceImpl|Implementation class for metastore service|
 *--+--+---+--+
-|42|lens.server.metrics.csv.directory.path|metrics/|Path of the directory in which to report metrics as separate csv files.|
+|42|lens.server.metastore.ws.resource.impl|org.apache.lens.server.metastore.MetastoreResource|Implementation class for Metastore Resource|
 *--+--+---+--+
-|43|lens.server.metrics.ganglia.host| |The ganglia host name|
+|43|lens.server.metrics.csv.directory.path|metrics/|Path of the directory in which to report metrics as separate csv files.|
 *--+--+---+--+
-|44|lens.server.metrics.ganglia.port| |The ganglia port|
+|44|lens.server.metrics.ganglia.host| |The ganglia host name|
 *--+--+---+--+
-|45|lens.server.metrics.graphite.host| |The graphite host name|
+|45|lens.server.metrics.ganglia.port| |The ganglia port|
 *--+--+---+--+
-|46|lens.server.metrics.graphite.port| |The graphite port|
+|46|lens.server.metrics.graphite.host| |The graphite host name|
 *--+--+---+--+
-|47|lens.server.metrics.reporting.period|10|The reporting period for metrics. The value is in seconds|
+|47|lens.server.metrics.graphite.port| |The graphite port|
 *--+--+---+--+
-|48|lens.server.mode|OPEN|The mode in which server should run. Allowed values are OPEN, READ_ONLY, METASTORE_READONLY, METASTORE_NODROP. OPEN mode will allow all requests. READ_ONLY mode will allow all requests on session resouce and only GET requests on all other resources. METASTORE_READONLY will allow GET on metastore and all other requests in other services. METASTORE_NODROP will not allow DELETE on metastore, will allow all other requests.|
+|48|lens.server.metrics.reporting.period|10|The reporting period for metrics. The value is in seconds|
 *--+--+---+--+
-|49|lens.server.moxyjson.ws.feature.impl|org.glassfish.jersey.moxy.json.MoxyJsonFeature|Enable Moxy json feature|
+|49|lens.server.mode|OPEN|The mode in which server should run. Allowed values are OPEN, READ_ONLY, METASTORE_READONLY, METASTORE_NODROP. OPEN mode will allow all requests. READ_ONLY mode will allow all requests on session resouce and only GET requests on all other resources. METASTORE_READONLY will allow GET on metastore and all other requests in other services. METASTORE_NODROP will not allow DELETE on metastore, will allow all other requests.|
 *--+--+---+--+
-|50|lens.server.moxyjsonconfigresovler.ws.feature.impl|org.apache.lens.api.util.MoxyJsonConfigurationContextResolver|Moxy json configuration resolver|
+|50|lens.server.moxyjson.ws.feature.impl|org.glassfish.jersey.moxy.json.MoxyJsonFeature|Enable Moxy json feature|
 *--+--+---+--+
-|51|lens.server.multipart.ws.feature.impl|org.glassfish.jersey.media.multipart.MultiPartFeature|Implementation class for query scheduler resource|
+|51|lens.server.moxyjsonconfigresovler.ws.feature.impl|org.apache.lens.api.util.MoxyJsonConfigurationContextResolver|Moxy json configuration resolver|
 *--+--+---+--+
-|52|lens.server.persist.location|file:///tmp/lensserver|The directory in which lens server will persist its state when it is going down. The location be on any Hadoop compatible file system. Server will read from the location when it is restarted and recovery is enabled. So, Server should have both read and write permissions to the location|
+|52|lens.server.multipart.ws.feature.impl|org.glassfish.jersey.media.multipart.MultiPartFeature|Implementation class for query scheduler resource|
 *--+--+---+--+
-|53|lens.server.query.acceptors| |Query Acceptors configured. Query acceptors are consulted first, before anything happens for the given query. They can either return null or return a messaging indicating why the given query shouldn't be accepted. These can be used to filter out queries at the earliest.|
+|53|lens.server.persist.location|file:///tmp/lensserver|The directory in which lens server will persist its state when it is going down. The location be on any Hadoop compatible file system. Server will read from the location when it is restarted and recovery is enabled. So, Server should have both read and write permissions to the location|
 *--+--+---+--+
-|54|lens.server.query.launching.constraint.factories|org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory|Factories used to instantiate constraints enforced on queries by lens. Every Factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.constraint.QueryLaunchingConstraint. A query will be launched only if all constraints pass.|
+|54|lens.server.query.acceptors| |Query Acceptors configured. Query acceptors are consulted first, before anything happens for the given query. They can either return null or return a messaging indicating why the given query shouldn't be accepted. These can be used to filter out queries at the earliest.|
 *--+--+---+--+
-|55|lens.server.query.phase1.rewriters| |Query phase 1 rewriters. This is to convert user query to cube query. The resulting cube query will be passed for validation and rewriting to hql query.\ |
+|55|lens.server.query.launching.constraint.factories|org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory|Factories used to instantiate constraints enforced on queries by lens. Every Factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.constraint.QueryLaunchingConstraint. A query will be launched only if all constraints pass.|
+*--+--+---+--+
+|56|lens.server.query.phase1.rewriters| |Query phase 1 rewriters. This is to convert user query to cube query. The resulting cube query will be passed for validation and rewriting to hql query.\ |
 |  |                                  | |Use cases will be to use extra intelligence to convert user query to optimized cube query.                                                              \ |
 |  |                                  | |Or define shortcuts for certain frequently used queries :)                                                                                                |
 *--+--+---+--+
-|56|lens.server.query.resultset.retention|1 day|Lens query resultset retention period. Default 1 day|
+|57|lens.server.query.resultset.retention|1 day|Lens query resultset retention period. Default 1 day|
 *--+--+---+--+
-|57|lens.server.query.service.impl|org.apache.lens.server.query.QueryExecutionServiceImpl|Implementation class for query execution service|
+|58|lens.server.query.service.impl|org.apache.lens.server.query.QueryExecutionServiceImpl|Implementation class for query execution service|
 *--+--+---+--+
-|58|lens.server.query.state.logger.enabled|true|Disable or enable the query state logger with this config. The location for the logger can be specified in logback xml for the class org.apache.lens.server.query.QueryExecutionServiceImpl.QueryStatusLogger|
+|59|lens.server.query.state.logger.enabled|true|Disable or enable the query state logger with this config. The location for the logger can be specified in logback xml for the class org.apache.lens.server.query.QueryExecutionServiceImpl.QueryStatusLogger|
 *--+--+---+--+
-|59|lens.server.query.ws.resource.impl|org.apache.lens.server.query.QueryServiceResource|Implementation class for Query Resource|
+|60|lens.server.query.ws.resource.impl|org.apache.lens.server.query.QueryServiceResource|Implementation class for Query Resource|
 *--+--+---+--+
-|60|lens.server.querypurger.sleep.interval|10000|The interval(milliseconds) with which purger to run periodically. Default 10 sec.|
+|61|lens.server.querypurger.sleep.interval|10000|The interval(milliseconds) with which purger to run periodically. Default 10 sec.|
 *--+--+---+--+
-|61|lens.server.quota.service.impl|org.apache.lens.server.quota.QuotaServiceImpl|Implementation class for quota service|
+|62|lens.server.quota.service.impl|org.apache.lens.server.quota.QuotaServiceImpl|Implementation class for quota service|
 *--+--+---+--+
-|62|lens.server.quota.ws.resource.impl|org.apache.lens.server.quota.QuotaResource|Implementation class for Quota Resource|
+|63|lens.server.quota.ws.resource.impl|org.apache.lens.server.quota.QuotaResource|Implementation class for Quota Resource|
 *--+--+---+--+
-|63|lens.server.recover.onrestart|true|If the flag is enabled, all the services will be started from last saved state, if disabled all the services will start afresh|
+|64|lens.server.recover.onrestart|true|If the flag is enabled, all the services will be started from last saved state, if disabled all the services will start afresh|
 *--+--+---+--+
-|64|lens.server.restart.enabled|true|If flag is enabled, all the services will be persisted to persistent location passed.|
+|65|lens.server.restart.enabled|true|If flag is enabled, all the services will be persisted to persistent location passed.|
 *--+--+---+--+
-|65|lens.server.resultset.purge.enabled|false|Whether to purge the query results|
+|66|lens.server.resultset.purge.enabled|false|Whether to purge the query results|
 *--+--+---+--+
-|66|lens.server.resultsetpurger.sleep.interval.secs|3600|Periodicity for Query result purger runs. Default 1 hour.|
+|67|lens.server.resultsetpurger.sleep.interval.secs|3600|Periodicity for Query result purger runs. Default 1 hour.|
 *--+--+---+--+
-|67|lens.server.savedquery.jdbc.dialectclass|org.apache.lens.server.query.save.SavedQueryDao$HSQLDialect|Dialect of the target DB, Default is HSQL. Override with the target DB used.|
+|68|lens.server.savedquery.jdbc.dialectclass|org.apache.lens.server.query.save.SavedQueryDao$HSQLDialect|Dialect of the target DB, Default is HSQL. Override with the target DB used.|
 *--+--+---+--+
-|68|lens.server.savedquery.list.default.count|20|Key denoting the default fetch value of saved query list api.|
+|69|lens.server.savedquery.list.default.count|20|Key denoting the default fetch value of saved query list api.|
 *--+--+---+--+
-|69|lens.server.savedquery.list.default.offset|0|Key denoting the default start value of saved query list api.|
+|70|lens.server.savedquery.list.default.offset|0|Key denoting the default start value of saved query list api.|
 *--+--+---+--+
-|70|lens.server.savedquery.service.impl|org.apache.lens.server.query.save.SavedQueryServiceImpl|Implementation class for saved query service|
+|71|lens.server.savedquery.service.impl|org.apache.lens.server.query.save.SavedQueryServiceImpl|Implementation class for saved query service|
 *--+--+---+--+
-|71|lens.server.savedquery.ws.resource.impl|org.apache.lens.server.query.save.SavedQueryResource|Implementation class for Saved query Resource|
+|72|lens.server.savedquery.ws.resource.impl|org.apache.lens.server.query.save.SavedQueryResource|Implementation class for Saved query Resource|
 *--+--+---+--+
-|72|lens.server.scheduler.service.impl|org.apache.lens.server.scheduler.SchedulerServiceImpl|Implementation class for query scheduler service|
+|73|lens.server.scheduler.service.impl|org.apache.lens.server.scheduler.SchedulerServiceImpl|Implementation class for query scheduler service|
 *--+--+---+--+
-|73|lens.server.scheduler.ws.resource.impl|org.apache.lens.server.scheduler.ScheduleResource|Implementation class for query scheduler resource|
+|74|lens.server.scheduler.ws.resource.impl|org.apache.lens.server.scheduler.ScheduleResource|Implementation class for query scheduler resource|
 *--+--+---+--+
-|74|lens.server.scheduling.queue.poll.interval.millisec|2000|The interval at which submission thread will poll scheduling queue to fetch the next query for submission. If value is less than equal to 0, then it would mean that thread will continuosly poll without sleeping. The interval has to be given in milliseconds.|
+|75|lens.server.scheduling.queue.poll.interval.millisec|2000|The interval at which submission thread will poll scheduling queue to fetch the next query for submission. If value is less than equal to 0, then it would mean that thread will continuosly poll without sleeping. The interval has to be given in milliseconds.|
 *--+--+---+--+
-|75|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
+|76|lens.server.serverMode.ws.filter.impl|org.apache.lens.server.ServerModeFilter|Implementation class for ServerMode Filter|
 *--+--+---+--+
-|76|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.|
+|77|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.|
 *--+--+---+--+
-|77|lens.server.servicenames|session,query,metastore,scheduler,quota|These services would be started in the specified order when lens-server starts up|
+|78|lens.server.servicenames|session,query,metastore,scheduler,quota|These services would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|78|lens.server.session.expiry.service.interval.secs|3600|Interval at which lens session expiry service runs|
+|79|lens.server.session.expiry.service.interval.secs|3600|Interval at which lens session expiry service runs|
 *--+--+---+--+
-|79|lens.server.session.service.impl|org.apache.lens.server.session.HiveSessionService|Implementation class for session service|
+|80|lens.server.session.service.impl|org.apache.lens.server.session.HiveSessionService|Implementation class for session service|
 *--+--+---+--+
-|80|lens.server.session.timeout.seconds|86400|Lens session timeout in seconds.If there is no activity on the session for this period then the session will be closed.Default timeout is one day.|
+|81|lens.server.session.timeout.seconds|86400|Lens session timeout in seconds.If there is no activity on the session for this period then the session will be closed.Default timeout is one day.|
 *--+--+---+--+
-|81|lens.server.session.ws.resource.impl|org.apache.lens.server.session.SessionResource|Implementation class for Session Resource|
+|82|lens.server.session.ws.resource.impl|org.apache.lens.server.session.SessionResource|Implementation class for Session Resource|
 *--+--+---+--+
-|82|lens.server.snapshot.interval|300000|Snapshot interval time in miliseconds for saving lens server state.|
+|83|lens.server.snapshot.interval|300000|Snapshot interval time in miliseconds for saving lens server state.|
 *--+--+---+--+
-|83|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.|
+|84|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.|
 *--+--+---+--+
-|84|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
+|85|lens.server.statistics.db|lensstats|Database to which statistics tables are created and partitions are added.|
 *--+--+---+--+
-|85|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
+|86|lens.server.statistics.log.rollover.interval|3600000|Default rate which log statistics store scans for rollups in milliseconds.|
 *--+--+---+--+
-|86|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
+|87|lens.server.statistics.store.class|org.apache.lens.server.stats.store.log.LogStatisticsStore|Default implementation of class used to persist Lens Statistics.|
 *--+--+---+--+
-|87|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
+|88|lens.server.statistics.warehouse.dir|file:///tmp/lens/statistics/warehouse|Default top level location where stats are moved by the log statistics store.|
 *--+--+---+--+
-|88|lens.server.total.query.cost.ceiling.per.user|-1.0|A query submitted by user will be launched only if total query cost of all current launched queries of user is less than or equal to total query cost ceiling defined by this property. This configuration value is only useful when TotalQueryCostCeilingConstraint is enabled by using org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory as one of the factories in lens.server.query.constraint.factories property. Default is -1.0 which means that there is no limit on the total query cost of launched queries submitted by a user.|
+|89|lens.server.total.query.cost.ceiling.per.user|-1.0|A query submitted by user will be launched only if total query cost of all current launched queries of user is less than or equal to total query cost ceiling defined by this property. This configuration value is only useful when TotalQueryCostCeilingConstraint is enabled by using org.apache.lens.server.query.constraint.TotalQueryCostCeilingConstraintFactory as one of the factories in lens.server.query.constraint.factories property. Default is -1.0 which means that there is no limit on the total query cost of launched queries submitted by a user.|
 *--+--+---+--+
-|89|lens.server.ui.base.uri|http://0.0.0.0:19999/|The base url for the Lens UI Server|
+|90|lens.server.ui.base.uri|http://0.0.0.0:19999/|The base url for the Lens UI Server|
 *--+--+---+--+
-|90|lens.server.ui.enable|true|Bringing up the ui server is optional. By default it brings up UI server.|
+|91|lens.server.ui.enable|true|Bringing up the ui server is optional. By default it brings up UI server.|
 *--+--+---+--+
-|91|lens.server.ui.enable.caching|true|Set this to false to disable static file caching in the UI server|
+|92|lens.server.ui.enable.caching|true|Set this to false to disable static file caching in the UI server|
 *--+--+---+--+
-|92|lens.server.ui.static.dir|webapp/lens-server/static|The base directory to server UI static files from|
+|93|lens.server.ui.static.dir|webapp/lens-server/static|The base directory to server UI static files from|
 *--+--+---+--+
-|93|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|
+|94|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|
 *--+--+---+--+
-|94|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.|
+|95|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.|
 *--+--+---+--+
-|95|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.|
+|96|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.|
 *--+--+---+--+
-|96|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.|
+|97|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.|
 *--+--+---+--+
-|97|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...|
+|98|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...|
 *--+--+---+--+
-|98|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
+|99|lens.server.user.resolver.ldap.bind.password| |Required for LDAP_BACKED_DATABASE user resolvers. ldap password for admin binding above|
 *--+--+---+--+
-|99|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.|
+|100|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.|
 *--+--+---+--+
-|100|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.|
+|101|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.|
 *--+--+---+--+
-|101|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|
+|102|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|
 *--+--+---+--+
-|102|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.|
+|103|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.|
 *--+--+---+--+
-|103|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...|
+|104|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...|
 *--+--+---+--+
-|104|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
+|105|lens.server.user.resolver.ldap.search.filter|(&(objectClass=user)(sAMAccountName=%s))|Required for LDAP_BACKED_DATABASE user resolvers. filter pattern for ldap search|
 *--+--+---+--+
-|105|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
+|106|lens.server.user.resolver.ldap.url| |Required for LDAP_BACKED_DATABASE user resolvers. ldap url to connect to.|
 *--+--+---+--+
-|106|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|
+|107|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|
 *--+--+---+--+
-|107|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
+|108|lens.server.user.resolver.type|FIXED|Type of user config resolver. allowed values are FIXED, PROPERTYBASED, DATABASE, LDAP_BACKED_DATABASE, CUSTOM.|
 *--+--+---+--+
-|108|lens.server.waiting.queries.selection.policy.factories|org.apache.lens.server.query.collect.UserSpecificWaitingQueriesSelectionPolicyFactory|Factories used to instantiate waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
+|109|lens.server.waiting.queries.selection.policy.factories|org.apache.lens.server.query.collect.UserSpecificWaitingQueriesSelectionPolicyFactory|Factories used to instantiate waiting queries selection policies. Every factory should be an implementation of org.apache.lens.server.api.common.ConfigBasedObjectCreationFactory and create an implementation of org.apache.lens.server.api.query.collect.WaitingQueriesSelectionPolicy.|
 *--+--+---+--+
-|109|lens.server.ws.featurenames|multipart,moxyjson,moxyjsonconfigresovler|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
+|110|lens.server.ws.featurenames|multipart,moxyjson,moxyjsonconfigresovler|These JAX-RS Feature(s) would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|110|lens.server.ws.filternames|authentication,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
+|111|lens.server.ws.filternames|authentication,consistentState,serverMode|These JAX-RS filters would be started in the specified order when lens-server starts up|
 *--+--+---+--+
-|111|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
+|112|lens.server.ws.listenernames|appevent|These listeners would be called in the specified order when lens-server starts up|
 *--+--+---+--+
-|112|lens.server.ws.resourcenames|session,metastore,query,quota,scheduler,index,log|These JAX-RS resources would be started in the specified order when lens-server starts up|
+|113|lens.server.ws.resourcenames|session,metastore,query,quota,scheduler,index,log|These JAX-RS resources would be started in the specified order when lens-server starts up|
 *--+--+---+--+
 The configuration parameters and their default values


[2/2] lens git commit: LENS-833 : Limit number of open sessions per user on session service

Posted by am...@apache.org.
LENS-833 : Limit number of open sessions per user on session service


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

Branch: refs/heads/master
Commit: 0ba17ef5c78bccf6699d696529b303879e27535b
Parents: 935647c
Author: Raju Bairishetti <ra...@apache.org>
Authored: Wed Mar 2 18:18:45 2016 +0530
Committer: Amareshwari Sriramadasu <am...@apache.org>
Committed: Wed Mar 2 18:18:45 2016 +0530

----------------------------------------------------------------------
 .../lens/api/error/ErrorCollectionFactory.java  |   8 +-
 .../org/apache/lens/api/error/LensError.java    |   8 +-
 .../apache/lens/api/error/LensHttpStatus.java   |  64 ++++++++
 .../apache/lens/api/result/LensAPIResult.java   |  13 +-
 lens-api/src/main/resources/lens-errors.conf    |   6 +
 .../lens/cli/TestLensConnectionCliCommands.java |  12 +-
 .../apache/lens/cli/TestLensCubeCommands.java   |  60 ++++----
 .../lens/cli/TestLensDatabaseCommands.java      |  18 ++-
 .../lens/cli/TestLensDimensionCommands.java     |   8 +
 .../cli/TestLensDimensionTableCommands.java     |  10 ++
 .../apache/lens/cli/TestLensFactCommands.java   |  11 ++
 .../TestLensFactCommandsWithMissingWeight.java  |  11 ++
 .../lens/cli/TestLensLogResourceCommands.java   |   7 +
 .../lens/cli/TestLensNativeTableCommands.java   |   4 +-
 .../lens/cli/TestLensStorageCommands.java       |   8 +
 .../lens/server/api/LensConfConstants.java      |   4 +
 .../lens/server/api/error/LensException.java    |   1 +
 .../org/apache/lens/server/BaseLensService.java | 149 ++++++++++++++-----
 .../lens/server/error/LensServerErrorCode.java  |   3 +-
 .../lens/server/session/SessionResource.java    |  12 +-
 .../src/main/resources/lensserver-default.xml   |   9 +-
 .../org/apache/lens/server/TestServerMode.java  |   3 +
 .../auth/FooBarAuthenticationProvider.java      |   3 +-
 .../common/ErrorResponseExpectedData.java       |   4 +-
 .../server/metastore/TestMetastoreService.java  |   1 +
 .../server/query/QueryAPIErrorResponseTest.java |   7 +-
 .../apache/lens/server/query/TestLensDAO.java   |   1 +
 .../server/session/TestSessionResource.java     |  85 ++++++++++-
 .../lens/server/ui/TestSessionUIResource.java   |  11 +-
 lens-server/src/test/resources/lens-site.xml    |   5 +
 src/site/apt/admin/config.apt                   | 148 +++++++++---------
 31 files changed, 513 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-api/src/main/java/org/apache/lens/api/error/ErrorCollectionFactory.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/error/ErrorCollectionFactory.java b/lens-api/src/main/java/org/apache/lens/api/error/ErrorCollectionFactory.java
index 741630b..28ea6c0 100644
--- a/lens-api/src/main/java/org/apache/lens/api/error/ErrorCollectionFactory.java
+++ b/lens-api/src/main/java/org/apache/lens/api/error/ErrorCollectionFactory.java
@@ -18,12 +18,12 @@
  */
 package org.apache.lens.api.error;
 
-import static javax.ws.rs.core.Response.Status;
-
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
+import javax.ws.rs.core.Response;
+
 import com.google.common.base.Optional;
 import com.google.common.collect.ImmutableMap;
 import com.typesafe.config.Config;
@@ -61,7 +61,9 @@ public class ErrorCollectionFactory {
 
       int errorCode = config.getInt(ERROR_CODE_KEY);
       int httpStatusCodeInt = config.getInt(HTTP_STATUS_CODE_KEY);
-      Status httpStatusCode = Status.fromStatusCode(httpStatusCodeInt);
+
+
+      Response.StatusType httpStatusCode = LensHttpStatus.fromStatusCode(httpStatusCodeInt);
       String errorMsg = config.getString(ERROR_MSG_KEY);
 
       Class payloadClass = null;

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-api/src/main/java/org/apache/lens/api/error/LensError.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/error/LensError.java b/lens-api/src/main/java/org/apache/lens/api/error/LensError.java
index 1cb7409..f49f104 100644
--- a/lens-api/src/main/java/org/apache/lens/api/error/LensError.java
+++ b/lens-api/src/main/java/org/apache/lens/api/error/LensError.java
@@ -19,10 +19,10 @@
 
 package org.apache.lens.api.error;
 
-import static javax.ws.rs.core.Response.Status;
-
 import static com.google.common.base.Preconditions.checkArgument;
 
+import javax.ws.rs.core.Response;
+
 import org.apache.commons.lang.StringUtils;
 
 import com.google.common.base.Optional;
@@ -41,11 +41,11 @@ import lombok.NonNull;
 public final class LensError {
 
   private final int errorCode;
-  private final Status httpStatusCode;
+  private final Response.StatusType httpStatusCode;
   private final String errorMsg;
   private final Optional<Class> payloadClass;
 
-  public LensError(final int errorCode, final Status httpStatusCode, final String errorMsg,
+  public LensError(final int errorCode, final Response.StatusType httpStatusCode, final String errorMsg,
       @NonNull final Optional<Class> payloadClass) {
 
     checkArgument(errorCode > 0);

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-api/src/main/java/org/apache/lens/api/error/LensHttpStatus.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/error/LensHttpStatus.java b/lens-api/src/main/java/org/apache/lens/api/error/LensHttpStatus.java
new file mode 100644
index 0000000..6da8e22
--- /dev/null
+++ b/lens-api/src/main/java/org/apache/lens/api/error/LensHttpStatus.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.lens.api.error;
+
+import javax.ws.rs.core.Response;
+
+import lombok.Getter;
+
+public enum LensHttpStatus implements Response.StatusType {
+  TOO_MANY_REQUESTS(429, "Too many requests");
+
+  @Getter private final int statusCode;
+  @Getter private final String reasonPhrase;
+  @Getter private final Response.Status.Family family;
+
+  private LensHttpStatus(int statusCode, String reasonPhrase) {
+    this.statusCode = statusCode;
+    this.reasonPhrase = reasonPhrase;
+    this.family = LensHttpStatus.familyOf(statusCode);
+  }
+
+  public String toString() {
+    return this.reasonPhrase;
+  }
+
+  public static Response.StatusType fromStatusCode(int statusCode) {
+    // Delegate all status code calls to Response.Status.
+    // Compute status code from LensHttpStatus only if it does not get status code from Status.
+    Response.StatusType httpStatusCode = Response.Status.fromStatusCode(statusCode);
+    if (httpStatusCode == null) {
+      LensHttpStatus[] arr = values();
+      int len = arr.length;
+
+      for (int i = 0; i < len; ++i) {
+        LensHttpStatus lensHttpStatus = arr[i];
+        if (lensHttpStatus.statusCode == statusCode) {
+          return lensHttpStatus;
+        }
+      }
+    }
+
+    return httpStatusCode;
+  }
+
+  public static Response.Status.Family familyOf(int statusCode) {
+    return Response.Status.Family.familyOf(statusCode);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java
----------------------------------------------------------------------
diff --git a/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java b/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java
index 238f9e6..a1664d6 100644
--- a/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java
+++ b/lens-api/src/main/java/org/apache/lens/api/result/LensAPIResult.java
@@ -21,6 +21,7 @@ package org.apache.lens.api.result;
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkState;
 
+import javax.ws.rs.core.Response;
 import javax.ws.rs.core.Response.Status;
 import javax.xml.bind.annotation.*;
 
@@ -59,7 +60,7 @@ public class LensAPIResult<DATA> {
   private LensErrorTO lensErrorTO;
 
   @XmlTransient
-  private Status httpStatusCode;
+  private Response.StatusType httpStatusCode;
 
   public static <DATA> LensAPIResult<DATA> composedOf(final String apiVersion,
       final String id, @NonNull final DATA data) {
@@ -67,20 +68,18 @@ public class LensAPIResult<DATA> {
   }
 
   public static <DATA> LensAPIResult<DATA> composedOf(final String apiVersion,
-      final String id, @NonNull final DATA data, @NonNull final Status httpStatusCode) {
-
+      final String id, @NonNull final DATA data, @NonNull final Response.StatusType httpStatusCode) {
     return new LensAPIResult<>(apiVersion, id, data, null, httpStatusCode);
   }
 
   public static LensAPIResult<NoResultData> composedOf(
       final String apiVersion, final String id, @NonNull final LensErrorTO lensErrorTO,
-      @NonNull final Status httpStatusCode) {
-
+      @NonNull final Response.StatusType httpStatusCode) {
     return new LensAPIResult<>(apiVersion, id, null, lensErrorTO, httpStatusCode);
   }
 
   private LensAPIResult(final String apiVersion, final String id, final DATA data, final LensErrorTO lensErrorTO,
-      @NonNull final Status httpStatusCode) {
+      @NonNull final Response.StatusType httpStatusCode) {
 
     /* The check commented below should be enabled in future, once story of apiVersion is clear. Right now there could
     be REST APIs throwing LensException without initializing apiVersion
@@ -100,7 +99,7 @@ public class LensAPIResult<DATA> {
     return (lensErrorTO != null) && lensErrorTO.areValidStackTracesPresent();
   }
 
-  public Status getHttpStatusCode() {
+  public Response.StatusType getHttpStatusCode() {
     return this.httpStatusCode;
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-api/src/main/resources/lens-errors.conf
----------------------------------------------------------------------
diff --git a/lens-api/src/main/resources/lens-errors.conf b/lens-api/src/main/resources/lens-errors.conf
index 09b221f..395d63b 100644
--- a/lens-api/src/main/resources/lens-errors.conf
+++ b/lens-api/src/main/resources/lens-errors.conf
@@ -26,6 +26,7 @@
 BAD_REQUEST = 400
 NOT_FOUND = 404
 UNAUTHORIZED = 401
+TOO_MANY_REQUESTS = 429
 INTERNAL_SERVER_ERROR = 500
 
 # Define all module specific errors
@@ -95,6 +96,11 @@ lensServerErrors = [
     payloadClass = org.apache.lens.api.query.SupportedQuerySubmitOperations
   }
 
+  {
+    errorCode = 2004
+    httpStatusCode = ${TOO_MANY_REQUESTS}
+    errorMsg = "Too many opened sessions for [%s] user. Session limit [%d] is already reached"
+  }
 ]
 
 # lensCubeErrors: Defined for lens-cube module

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java
index 558e97f..3c4d320 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensConnectionCliCommands.java
@@ -337,9 +337,13 @@ public class TestLensConnectionCliCommands extends LensCliApplicationTest {
     LensClient client = new LensClient();
     LensConnectionCommands commands = new LensConnectionCommands();
     commands.setClient(client);
-    LensSessionHandle sessionHandle = client.getConnection().getSessionHandle();
-    Assert.assertNotNull(sessionHandle);
-    String output = commands.getSessionHandle();
-    Assert.assertTrue(output.contains(sessionHandle.getPublicId().toString()), "session handle output: " + output);
+    try {
+      LensSessionHandle sessionHandle = client.getConnection().getSessionHandle();
+      Assert.assertNotNull(sessionHandle);
+      String output = commands.getSessionHandle();
+      Assert.assertTrue(output.contains(sessionHandle.getPublicId().toString()), "session handle output: " + output);
+    } finally {
+      commands.quitShell();
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensCubeCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensCubeCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensCubeCommands.java
index 97ca2c8..5a353df 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensCubeCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensCubeCommands.java
@@ -52,36 +52,40 @@ public class TestLensCubeCommands extends LensCliApplicationTest {
   @Test
   public void testCubeCommands() throws Exception {
     LensClient client = new LensClient();
-    LensDimensionCommands dimensionCommand = new LensDimensionCommands();
-    dimensionCommand.setClient(client);
-    dimensionCommand.createDimension(new File(
-      TestLensCubeCommands.class.getClassLoader().getResource("test-detail.xml").toURI()));
-    dimensionCommand.createDimension(new File(
-      TestLensCubeCommands.class.getClassLoader().getResource("test-dimension.xml").toURI()));
-    LensCubeCommands command = new LensCubeCommands();
-    command.setClient(client);
-    LOG.debug("Starting to test cube commands");
-    URL cubeSpec = TestLensCubeCommands.class.getClassLoader().getResource("sample-cube.xml");
-    String cubeList = command.showCubes();
-    assertFalse(cubeList.contains("sample_cube"));
-    command.createCube(new File(cubeSpec.toURI()));
-    cubeList = command.showCubes();
-    assertEquals(command.getLatest("sample_cube", "dt"), "No Data Available");
-    assertTrue(cubeList.contains("sample_cube"));
-    testJoinChains(command);
-    testFields(command);
-    testUpdateCommand(new File(cubeSpec.toURI()), command);
-    command.dropCube("sample_cube");
     try {
-      command.getLatest("sample_cube", "dt");
-      fail("should have failed as cube doesn't exist");
-    } catch (Exception e) {
-      //pass
+      LensDimensionCommands dimensionCommand = new LensDimensionCommands();
+      dimensionCommand.setClient(client);
+      dimensionCommand.createDimension(new File(
+          TestLensCubeCommands.class.getClassLoader().getResource("test-detail.xml").toURI()));
+      dimensionCommand.createDimension(new File(
+          TestLensCubeCommands.class.getClassLoader().getResource("test-dimension.xml").toURI()));
+      LensCubeCommands command = new LensCubeCommands();
+      command.setClient(client);
+      LOG.debug("Starting to test cube commands");
+      URL cubeSpec = TestLensCubeCommands.class.getClassLoader().getResource("sample-cube.xml");
+      String cubeList = command.showCubes();
+      assertFalse(cubeList.contains("sample_cube"));
+      command.createCube(new File(cubeSpec.toURI()));
+      cubeList = command.showCubes();
+      assertEquals(command.getLatest("sample_cube", "dt"), "No Data Available");
+      assertTrue(cubeList.contains("sample_cube"));
+      testJoinChains(command);
+      testFields(command);
+      testUpdateCommand(new File(cubeSpec.toURI()), command);
+      command.dropCube("sample_cube");
+      try {
+        command.getLatest("sample_cube", "dt");
+        fail("should have failed as cube doesn't exist");
+      } catch (Exception e) {
+        //pass
+      }
+      cubeList = command.showCubes();
+      assertFalse(cubeList.contains("sample_cube"));
+      dimensionCommand.dropDimension("test_detail");
+      dimensionCommand.dropDimension("test_dim");
+    } finally {
+      client.closeConnection();
     }
-    cubeList = command.showCubes();
-    assertFalse(cubeList.contains("sample_cube"));
-    dimensionCommand.dropDimension("test_detail");
-    dimensionCommand.dropDimension("test_dim");
   }
 
   private void testJoinChains(LensCubeCommands command) {

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensDatabaseCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensDatabaseCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensDatabaseCommands.java
index 32ed7b0..705aace 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensDatabaseCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensDatabaseCommands.java
@@ -45,13 +45,17 @@ public class TestLensDatabaseCommands extends LensCliApplicationTest {
   @Test
   public void testDatabaseCommands() throws URISyntaxException {
     LensClient client = new LensClient();
-    LensDatabaseCommands command = new LensDatabaseCommands();
-    LensCubeCommands cubeCommand = new LensCubeCommands();
-    command.setClient(client);
-    cubeCommand.setClient(client);
-    boolean cascade = true;
-    for(int i = 0; i < 4; i++, cascade = !cascade) {
-      testDrop(command, cubeCommand, cascade);
+    try {
+      LensDatabaseCommands command = new LensDatabaseCommands();
+      LensCubeCommands cubeCommand = new LensCubeCommands();
+      command.setClient(client);
+      cubeCommand.setClient(client);
+      boolean cascade = true;
+      for (int i = 0; i < 4; i++, cascade = !cascade) {
+        testDrop(command, cubeCommand, cascade);
+      }
+    } finally {
+      client.closeConnection();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java
index 955d4c7..a3f01c3 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionCommands.java
@@ -33,6 +33,7 @@ import org.apache.lens.cli.table.XJoinChainTable;
 import org.apache.lens.client.LensClient;
 
 import org.testng.Assert;
+import org.testng.annotations.AfterTest;
 import org.testng.annotations.Test;
 
 import lombok.extern.slf4j.Slf4j;
@@ -55,6 +56,13 @@ public class TestLensDimensionCommands extends LensCliApplicationTest {
     return command;
   }
 
+  @AfterTest
+  public void cleanUp() {
+    if (command != null) {
+      command.getClient().closeConnection();
+    }
+  }
+
   /**
    * Creates the dimension.
    *

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionTableCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionTableCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionTableCommands.java
index bea128b..5d6d768 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionTableCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensDimensionTableCommands.java
@@ -30,6 +30,7 @@ import org.apache.lens.cli.commands.LensDimensionCommands;
 import org.apache.lens.cli.commands.LensDimensionTableCommands;
 import org.apache.lens.client.LensClient;
 
+import org.testng.annotations.AfterTest;
 import org.testng.annotations.Test;
 
 import lombok.extern.slf4j.Slf4j;
@@ -65,6 +66,15 @@ public class TestLensDimensionTableCommands extends LensCliApplicationTest {
     return dimensionCommand;
   }
 
+  @AfterTest
+  public void cleanUp() {
+    if (command != null) {
+      command.getClient().closeConnection();
+    }
+    if (dimensionCommand != null) {
+      dimensionCommand.getClient().closeConnection();
+    }
+  }
 
   /**
    * Test dim table commands.

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommands.java
index 1454e5f..9670d8f 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommands.java
@@ -31,6 +31,7 @@ import org.apache.lens.cli.commands.LensCubeCommands;
 import org.apache.lens.cli.commands.LensFactCommands;
 import org.apache.lens.client.LensClient;
 
+import org.testng.annotations.AfterTest;
 import org.testng.annotations.Test;
 
 import lombok.extern.slf4j.Slf4j;
@@ -95,6 +96,16 @@ public class TestLensFactCommands extends LensCliApplicationTest {
     return cubeCommands;
   }
 
+  @AfterTest
+  public void cleanUp() {
+    if (command != null) {
+      command.getClient().closeConnection();
+    }
+    if (cubeCommands != null) {
+      cubeCommands.getClient().closeConnection();
+    }
+  }
+
   /**
    * Adds the fact1 table.
    *

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommandsWithMissingWeight.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommandsWithMissingWeight.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommandsWithMissingWeight.java
index 9fce233..24f9279 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommandsWithMissingWeight.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensFactCommandsWithMissingWeight.java
@@ -31,6 +31,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.testng.Assert;
+import org.testng.annotations.AfterTest;
 import org.testng.annotations.Test;
 
 
@@ -108,6 +109,16 @@ public class TestLensFactCommandsWithMissingWeight extends LensCliApplicationTes
     return cubeCommands;
   }
 
+  @AfterTest
+  public void cleanUp() {
+    if (command != null) {
+      command.getClient().closeConnection();
+    }
+    if (cubeCommands != null) {
+      cubeCommands.getClient().closeConnection();
+    }
+  }
+
   /**
    * Adds the fact_without_wt table.
    *

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensLogResourceCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensLogResourceCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensLogResourceCommands.java
index f4b043e..72a5d9d 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensLogResourceCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensLogResourceCommands.java
@@ -29,6 +29,7 @@ import org.apache.lens.client.LensClient;
 import org.apache.commons.io.FileUtils;
 
 import org.testng.Assert;
+import org.testng.annotations.AfterTest;
 import org.testng.annotations.BeforeTest;
 import org.testng.annotations.Test;
 
@@ -44,6 +45,12 @@ public class TestLensLogResourceCommands extends LensCliApplicationTest {
     commands.setClient(client);
   }
 
+  @AfterTest
+  public void cleanup() {
+    if (client != null) {
+      client.closeConnection();
+    }
+  }
   @Test
   public void testLogResourceFileDoesNotExist() throws IOException {
     // check for 404 response

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensNativeTableCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensNativeTableCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensNativeTableCommands.java
index e5f11f2..80dd0d1 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensNativeTableCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensNativeTableCommands.java
@@ -45,8 +45,8 @@ public class TestLensNativeTableCommands extends LensCliApplicationTest {
    */
   @Test
   public void testNativeTableCommands() throws Exception {
+    LensClient client = new LensClient();
     try {
-      LensClient client = new LensClient();
       LensNativeTableCommands command = new LensNativeTableCommands();
       command.setClient(client);
       LOG.debug("Starting to test nativetable commands");
@@ -64,7 +64,7 @@ public class TestLensNativeTableCommands extends LensCliApplicationTest {
       Assert.assertTrue(desc.contains("test.hive.table.prop"));
     } finally {
       LensServerTestUtil.dropHiveTable("test_native_table_command");
-
+      client.closeConnection();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-cli/src/test/java/org/apache/lens/cli/TestLensStorageCommands.java
----------------------------------------------------------------------
diff --git a/lens-cli/src/test/java/org/apache/lens/cli/TestLensStorageCommands.java b/lens-cli/src/test/java/org/apache/lens/cli/TestLensStorageCommands.java
index a178296..c4ab614 100644
--- a/lens-cli/src/test/java/org/apache/lens/cli/TestLensStorageCommands.java
+++ b/lens-cli/src/test/java/org/apache/lens/cli/TestLensStorageCommands.java
@@ -27,6 +27,7 @@ import org.apache.lens.client.LensClient;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.testng.Assert;
+import org.testng.annotations.AfterTest;
 import org.testng.annotations.Test;
 
 /**
@@ -61,6 +62,13 @@ public class TestLensStorageCommands extends LensCliApplicationTest {
     return command;
   }
 
+  @AfterTest
+  public void cleanup() {
+    if (command != null) {
+      command.getClient().closeConnection();
+    }
+  }
+
   /**
    * Drop storage.
    *

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/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 1b7d0f9..52a7ccc 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
@@ -89,6 +89,10 @@ public final class LensConfConstants {
    */
   public static final String WS_FEATURE_NAMES = SERVER_PFX + "ws.featurenames";
 
+  public static final String MAX_SESSIONS_PER_USER = SERVER_PFX + "max.sessions.per.user";
+
+  public static final Integer DEFAULT_MAX_SESSIONS_PER_USER = 10;
+
   /**
    * The Constant SERVICE_IMPL_SFX.
    */

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
----------------------------------------------------------------------
diff --git a/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java b/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
index a1ffeb6..6f8b175 100644
--- a/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
+++ b/lens-server-api/src/main/java/org/apache/lens/server/api/error/LensException.java
@@ -185,6 +185,7 @@ public class LensException extends Exception implements Comparable<LensException
     final String apiVersion, final String id) {
 
     final LensError lensError = errorCollection.getLensError(getErrorCode());
+
     final LensErrorTO lensErrorTO = buildLensErrorTO(errorCollection, lensError);
     lensAPIResult = LensAPIResult.composedOf(apiVersion, id, lensErrorTO, lensError.getHttpStatusCode());
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java b/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java
index 0821fe7..be31cd8 100644
--- a/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java
+++ b/lens-server/src/main/java/org/apache/lens/server/BaseLensService.java
@@ -41,6 +41,7 @@ import org.apache.lens.server.api.error.LensException;
 import org.apache.lens.server.api.events.LensEvent;
 import org.apache.lens.server.api.events.LensEventService;
 import org.apache.lens.server.api.health.HealthStatus;
+import org.apache.lens.server.error.LensServerErrorCode;
 import org.apache.lens.server.session.LensSessionImpl;
 import org.apache.lens.server.user.UserConfigLoaderFactory;
 import org.apache.lens.server.util.UtilityMethods;
@@ -49,7 +50,6 @@ import org.apache.commons.lang3.StringUtils;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-
 import org.apache.hive.service.CompositeService;
 import org.apache.hive.service.auth.AuthenticationProviderFactory;
 import org.apache.hive.service.auth.HiveAuthFactory;
@@ -82,7 +82,22 @@ public abstract class BaseLensService extends CompositeService implements Extern
   // lens session before submitting a query to hive server
   /** The session map. */
   protected static final ConcurrentHashMap<String, LensSessionHandle> SESSION_MAP
-    = new ConcurrentHashMap<String, LensSessionHandle>();
+    = new ConcurrentHashMap<>();
+
+  /**
+   * This map maintains active session count for each user
+   * Key: userName
+   * Value: number of sessions opened
+   */
+  private static final Map<String, Integer> SESSIONS_PER_USER = new ConcurrentHashMap<>();
+
+  /**
+   * Maintains a map with user to SessionUser instance.
+   * This map is used for acquiring a lock on specific user for while opening & closing sessions
+   */
+  private static final Map<String, SessionUser> SESSION_USER_INSTANCE_MAP = new HashMap<>();
+
+  private final int maxNumSessionsPerUser;
 
   /**
    * Instantiates a new lens service.
@@ -93,6 +108,15 @@ public abstract class BaseLensService extends CompositeService implements Extern
   protected BaseLensService(String name, CLIService cliService) {
     super(name);
     this.cliService = cliService;
+    maxNumSessionsPerUser = getMaximumNumberOfSessionsPerUser();
+  }
+
+  private static class SessionUser {
+    private String sessionUser;
+
+    public SessionUser(String user) {
+      this.sessionUser = user;
+    }
   }
 
   /**
@@ -110,6 +134,16 @@ public abstract class BaseLensService extends CompositeService implements Extern
     return BaseLensService.SESSION_MAP.size();
   }
 
+  private static int getMaximumNumberOfSessionsPerUser() {
+    return LensServerConf.getHiveConf().getInt(LensConfConstants.MAX_SESSIONS_PER_USER,
+      LensConfConstants.DEFAULT_MAX_SESSIONS_PER_USER);
+  }
+
+  private boolean isMaxSessionsLimitReachedPerUser(String userName) {
+    Integer numSessions = SESSIONS_PER_USER.get(userName);
+    return numSessions != null && numSessions >= maxNumSessionsPerUser;
+  }
+
   /**
    * Open session.
    *
@@ -127,42 +161,66 @@ public abstract class BaseLensService extends CompositeService implements Extern
     SessionHandle sessionHandle;
     username = UtilityMethods.removeDomain(username);
     doPasswdAuth(username, password);
-    try {
-      Map<String, String> sessionConf = new HashMap<String, String>();
-      sessionConf.putAll(LensSessionImpl.DEFAULT_HIVE_SESSION_CONF);
-      if (configuration != null) {
-        sessionConf.putAll(configuration);
-      }
-      Map<String, String> userConfig = UserConfigLoaderFactory.getUserConfig(username);
-      log.info("Got user config: {}", userConfig);
-      UtilityMethods.mergeMaps(sessionConf, userConfig, false);
-      sessionConf.put(LensConfConstants.SESSION_LOGGEDIN_USER, username);
-      if (sessionConf.get(LensConfConstants.SESSION_CLUSTER_USER) == null) {
-        log.info("Didn't get cluster user from user config loader. Setting same as logged in user: {}", username);
-        sessionConf.put(LensConfConstants.SESSION_CLUSTER_USER, username);
+    SessionUser sessionUser = SESSION_USER_INSTANCE_MAP.get(username);
+    if (sessionUser == null) {
+      sessionUser = new SessionUser(username);
+      SESSION_USER_INSTANCE_MAP.put(username, sessionUser);
+    }
+    synchronized (sessionUser) {
+      if (isMaxSessionsLimitReachedPerUser(username)) {
+        log.error("Can not open new session as session limit {} is reached already for {} user",
+            maxNumSessionsPerUser, username);
+        throw new LensException(LensServerErrorCode.TOO_MANY_OPEN_SESSIONS.getLensErrorInfo(), username,
+            maxNumSessionsPerUser);
       }
-      String clusterUser = sessionConf.get(LensConfConstants.SESSION_CLUSTER_USER);
-      password = "useless";
-      if (cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION)
-        .equals(HiveAuthFactory.AuthTypes.KERBEROS.toString())
-        && cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
-        String delegationTokenStr = null;
-        try {
-          delegationTokenStr = cliService.getDelegationTokenFromMetaStore(username);
-        } catch (UnsupportedOperationException e) {
-          // The delegation token is not applicable in the given deployment mode
+      try {
+        Map<String, String> sessionConf = new HashMap<String, String>();
+        sessionConf.putAll(LensSessionImpl.DEFAULT_HIVE_SESSION_CONF);
+        if (configuration != null) {
+          sessionConf.putAll(configuration);
         }
-        sessionHandle = cliService.openSessionWithImpersonation(clusterUser, password, sessionConf, delegationTokenStr);
-      } else {
-        sessionHandle = cliService.openSession(clusterUser, password, sessionConf);
+        Map<String, String> userConfig = UserConfigLoaderFactory.getUserConfig(username);
+        log.info("Got user config: {}", userConfig);
+        UtilityMethods.mergeMaps(sessionConf, userConfig, false);
+        sessionConf.put(LensConfConstants.SESSION_LOGGEDIN_USER, username);
+        if (sessionConf.get(LensConfConstants.SESSION_CLUSTER_USER) == null) {
+          log.info("Didn't get cluster user from user config loader. Setting same as logged in user: {}", username);
+          sessionConf.put(LensConfConstants.SESSION_CLUSTER_USER, username);
+        }
+        String clusterUser = sessionConf.get(LensConfConstants.SESSION_CLUSTER_USER);
+        password = "useless";
+        if (cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION)
+            .equals(HiveAuthFactory.AuthTypes.KERBEROS.toString())
+            && cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS)) {
+          String delegationTokenStr = null;
+          try {
+            delegationTokenStr = cliService.getDelegationTokenFromMetaStore(username);
+          } catch (UnsupportedOperationException e) {
+            // The delegation token is not applicable in the given deployment mode
+          }
+          sessionHandle = cliService.openSessionWithImpersonation(clusterUser, password, sessionConf,
+              delegationTokenStr);
+        } else {
+          sessionHandle = cliService.openSession(clusterUser, password, sessionConf);
+        }
+      } catch (Exception e) {
+        throw new LensException(e);
       }
-    } catch (Exception e) {
-      throw new LensException(e);
+      LensSessionHandle lensSessionHandle = new LensSessionHandle(sessionHandle.getHandleIdentifier().getPublicId(),
+          sessionHandle.getHandleIdentifier().getSecretId());
+      SESSION_MAP.put(lensSessionHandle.getPublicId().toString(), lensSessionHandle);
+      updateSessionsPerUser(username);
+      return lensSessionHandle;
+    }
+  }
+
+  private void updateSessionsPerUser(String userName) {
+    Integer numOfSessions = SESSIONS_PER_USER.get(userName);
+    if (null == numOfSessions) {
+      SESSIONS_PER_USER.put(userName, 1);
+    } else {
+      SESSIONS_PER_USER.put(userName, ++numOfSessions);
     }
-    LensSessionHandle lensSession = new LensSessionHandle(sessionHandle.getHandleIdentifier().getPublicId(),
-      sessionHandle.getHandleIdentifier().getSecretId());
-    SESSION_MAP.put(lensSession.getPublicId().toString(), lensSession);
-    return lensSession;
   }
 
   protected LensEventService getEventService() {
@@ -194,6 +252,7 @@ public abstract class BaseLensService extends CompositeService implements Extern
       LensSessionHandle restoredSession = new LensSessionHandle(restoredHandle.getHandleIdentifier().getPublicId(),
         restoredHandle.getHandleIdentifier().getSecretId());
       SESSION_MAP.put(restoredSession.getPublicId().toString(), restoredSession);
+      updateSessionsPerUser(userName);
     } catch (HiveSQLException e) {
       throw new LensException("Error restoring session " + sessionHandle, e);
     }
@@ -236,13 +295,33 @@ public abstract class BaseLensService extends CompositeService implements Extern
    */
   public void closeSession(LensSessionHandle sessionHandle) throws LensException {
     try {
+      String userName = getSession(sessionHandle).getLoggedInUser();
       cliService.closeSession(getHiveSessionHandle(sessionHandle));
-      SESSION_MAP.remove(sessionHandle.getPublicId().toString());
+      String publicId = sessionHandle.getPublicId().toString();
+      SESSION_MAP.remove(publicId);
+      decrementSessionCountForUser(sessionHandle, userName);
     } catch (Exception e) {
       throw new LensException(e);
     }
   }
 
+  private void decrementSessionCountForUser(LensSessionHandle sessionHandle, String userName) {
+    SessionUser sessionUser = SESSION_USER_INSTANCE_MAP.get(userName);
+    if (sessionUser == null) {
+      log.info("Trying to close invalid session {} for user {}", sessionHandle, userName);
+      return;
+    }
+    synchronized (sessionUser) {
+      Integer sessionCount = SESSIONS_PER_USER.get(userName);
+      log.info("Closed session {} for {} user", sessionHandle, userName);
+      if (sessionCount == 1) {
+        SESSIONS_PER_USER.remove(userName);
+      } else {
+        SESSIONS_PER_USER.put(userName, --sessionCount);
+      }
+    }
+  }
+
   public SessionManager getSessionManager() {
     return cliService.getSessionManager();
   }

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java b/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java
index dc20f0f..2443fec 100644
--- a/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java
+++ b/lens-server/src/main/java/org/apache/lens/server/error/LensServerErrorCode.java
@@ -24,7 +24,8 @@ public enum LensServerErrorCode {
 
   SESSION_ID_NOT_PROVIDED(2001, 0),
   NULL_OR_EMPTY_OR_BLANK_QUERY(2002, 0),
-  UNSUPPORTED_QUERY_SUBMIT_OPERATION(2003, 0);
+  UNSUPPORTED_QUERY_SUBMIT_OPERATION(2003, 0),
+  TOO_MANY_OPEN_SESSIONS(2004, 0);
 
   public LensErrorInfo getLensErrorInfo() {
     return this.errorInfo;

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java b/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
index ac77418..0daa286 100644
--- a/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
+++ b/lens-server/src/main/java/org/apache/lens/server/session/SessionResource.java
@@ -24,12 +24,14 @@ import java.util.Map;
 
 import javax.ws.rs.*;
 import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
 
 import org.apache.lens.api.APIResult;
 import org.apache.lens.api.APIResult.Status;
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.StringList;
+import org.apache.lens.api.error.ErrorCollection;
 import org.apache.lens.server.BaseLensService;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.error.LensException;
@@ -52,6 +54,8 @@ public class SessionResource {
   /** The session service. */
   private SessionService sessionService;
 
+  private final ErrorCollection errorCollection;
+
   /**
    * API to know if session service is up and running
    *
@@ -70,6 +74,7 @@ public class SessionResource {
    */
   public SessionResource() throws LensException {
     sessionService = LensServices.get().getService(SessionService.NAME);
+    errorCollection = LensServices.get().getErrorCollection();
   }
 
   /**
@@ -87,7 +92,7 @@ public class SessionResource {
   public LensSessionHandle openSession(@FormDataParam("username") String username,
     @FormDataParam("password") String password,
     @FormDataParam("database")  @DefaultValue("") String database,
-    @FormDataParam("sessionconf") LensConf sessionconf) {
+    @FormDataParam("sessionconf") LensConf sessionconf) throws LensException {
     try {
       Map<String, String> conf;
       if (sessionconf != null) {
@@ -97,7 +102,10 @@ public class SessionResource {
       }
       return sessionService.openSession(username, password, database,   conf);
     } catch (LensException e) {
-      throw new WebApplicationException(e);
+      e.buildLensErrorResponse(errorCollection, null,
+          LensServices.get().getLogSegregationContext().getLogSegragationId());
+      Response response = Response.status(e.getLensAPIResult().getHttpStatusCode()).build();
+      throw new WebApplicationException(response);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/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 a711d03..6bb66d9 100644
--- a/lens-server/src/main/resources/lensserver-default.xml
+++ b/lens-server/src/main/resources/lensserver-default.xml
@@ -826,5 +826,12 @@
     <value>20</value>
     <description>Key denoting the default fetch value of saved query list api.</description>
   </property>
-
+  <property>
+    <name>lens.server.max.sessions.per.user</name>
+    <value>10</value>
+    <description>Number of sessions can be allowed for each user.
+      User has to close one of the active sessions to open a new session once limit is reached. Otherwise Server throws
+      an exception by saying that opened session limit has been already reached for user.
+    </description>
+  </property>
 </configuration>

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java b/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java
index caf968a..fce6e5f 100644
--- a/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java
+++ b/lens-server/src/test/java/org/apache/lens/server/TestServerMode.java
@@ -53,6 +53,8 @@ import org.testng.annotations.Test;
 @Test(alwaysRun = true, groups = "filter-test", dependsOnGroups = "restart-test")
 public class TestServerMode extends LensAllApplicationJerseyTest {
 
+  private LensSessionHandle lensSessionHandle;
+
   /*
    * (non-Javadoc)
    *
@@ -72,6 +74,7 @@ public class TestServerMode extends LensAllApplicationJerseyTest {
    */
   @AfterTest
   public void tearDown() throws Exception {
+    RestAPITestUtil.closeSession(target(), lensSessionHandle, defaultMT);
     super.tearDown();
   }
 

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/java/org/apache/lens/server/auth/FooBarAuthenticationProvider.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/auth/FooBarAuthenticationProvider.java b/lens-server/src/test/java/org/apache/lens/server/auth/FooBarAuthenticationProvider.java
index 8e22837..d4c3a83 100644
--- a/lens-server/src/test/java/org/apache/lens/server/auth/FooBarAuthenticationProvider.java
+++ b/lens-server/src/test/java/org/apache/lens/server/auth/FooBarAuthenticationProvider.java
@@ -31,7 +31,8 @@ public class FooBarAuthenticationProvider implements PasswdAuthenticationProvide
   public static final String MSG = "<username,password>!=<foo@localhost,bar>";
 
   /** The allowed combinations. */
-  private final String[][] allowedCombinations = new String[][]{{"foo", "bar"}, {"anonymous", ""}};
+  private final String[][] allowedCombinations
+    = new String[][]{{"foo", "bar"}, {"anonymous", ""}, {"test", "test"}, {"UITest", "UITest"}};
 
   /*
    * (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java b/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java
index 4d4f137..3ede853 100644
--- a/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java
+++ b/lens-server/src/test/java/org/apache/lens/server/common/ErrorResponseExpectedData.java
@@ -30,10 +30,10 @@ import org.apache.lens.api.result.LensErrorTO;
 
 public class ErrorResponseExpectedData {
 
-  private final Response.Status expectedStatus;
+  private final Response.StatusType expectedStatus;
   private final LensErrorTO expectedLensErrorTO;
 
-  public ErrorResponseExpectedData(final Response.Status expectedStatus,
+  public ErrorResponseExpectedData(final Response.StatusType expectedStatus,
       final LensErrorTO expectedLensErrorTO) {
 
     this.expectedStatus = expectedStatus;

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java b/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
index a541e9b..3b091f5 100644
--- a/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
+++ b/lens-server/src/test/java/org/apache/lens/server/metastore/TestMetastoreService.java
@@ -61,6 +61,7 @@ import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hadoop.mapred.SequenceFileInputFormat;
 
 import org.glassfish.jersey.test.TestProperties;
+
 import org.testng.Assert;
 import org.testng.annotations.*;
 

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java b/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java
index 30f7aff..29fcd6d 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/QueryAPIErrorResponseTest.java
@@ -121,6 +121,7 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);
 
     expectedData.verify(response);
+    closeSession(target(), sessionId, mt);
   }
 
   @Test(dataProvider = "mediaTypeData")
@@ -140,6 +141,7 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);
 
     expectedData.verify(response);
+    closeSession(target(), sessionId, mt);
   }
 
   @Test(dataProvider = "mediaTypeData")
@@ -164,7 +166,9 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
 
     assertTrue(expectedLensErrorTO1.getMessage().equals(responseLensErrorTO.getMessage())
             || expectedLensErrorTO2.getMessage().equals(responseLensErrorTO.getMessage()),
-      "Message is " + responseLensErrorTO.getMessage());
+        "Message is " + responseLensErrorTO.getMessage());
+    closeSession(target(), sessionId, mt);
+
   }
 
   @Test(dataProvider = "mediaTypeData")
@@ -180,6 +184,7 @@ public class QueryAPIErrorResponseTest extends LensJerseyTest {
     ErrorResponseExpectedData expectedData = new ErrorResponseExpectedData(BAD_REQUEST, expectedLensErrorTO);
 
     expectedData.verify(response);
+    closeSession(target(), sessionId, mt);
   }
 
   @Test(dataProvider = "mediaTypeData")

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java b/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java
index a5ee5cc..760e306 100644
--- a/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java
+++ b/lens-server/src/test/java/org/apache/lens/server/query/TestLensDAO.java
@@ -142,5 +142,6 @@ public class TestLensDAO {
       Long.MAX_VALUE);
     Assert.assertEquals(daoTestQueryHandles.size(), 1);
     Assert.assertEquals(daoTestQueryHandles.get(0).getHandleId().toString(), finishedHandle);
+    service.closeSession(session);
   }
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/java/org/apache/lens/server/session/TestSessionResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/session/TestSessionResource.java b/lens-server/src/test/java/org/apache/lens/server/session/TestSessionResource.java
index bde7b9b..e54f0af 100644
--- a/lens-server/src/test/java/org/apache/lens/server/session/TestSessionResource.java
+++ b/lens-server/src/test/java/org/apache/lens/server/session/TestSessionResource.java
@@ -22,7 +22,9 @@ import java.io.File;
 import java.io.IOException;
 import java.net.URL;
 import java.net.URLClassLoader;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 
 import javax.ws.rs.ClientErrorException;
 import javax.ws.rs.NotFoundException;
@@ -38,6 +40,7 @@ import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.api.StringList;
 import org.apache.lens.server.LensJerseyTest;
+import org.apache.lens.server.LensServerConf;
 import org.apache.lens.server.LensServices;
 import org.apache.lens.server.api.LensConfConstants;
 import org.apache.lens.server.api.error.LensException;
@@ -45,6 +48,7 @@ import org.apache.lens.server.api.metrics.MetricsService;
 import org.apache.lens.server.api.session.SessionService;
 import org.apache.lens.server.common.LenServerTestException;
 import org.apache.lens.server.common.LensServerTestFileUtils;
+import org.apache.lens.server.common.RestAPITestUtil;
 import org.apache.lens.server.common.TestResourceFile;
 
 import org.apache.commons.io.FileUtils;
@@ -80,6 +84,7 @@ public class TestSessionResource extends LensJerseyTest {
   @BeforeTest
   public void setUp() throws Exception {
     metricsSvc = LensServices.get().getService(MetricsService.NAME);
+    LensServices.get().getLogSegregationContext().setLogSegregationId("logid");
     super.setUp();
   }
 
@@ -177,7 +182,7 @@ public class TestSessionResource extends LensJerseyTest {
     // Create another session
     final LensSessionHandle handle2 = target.request(mt).post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
       LensSessionHandle.class);
-    Assert.assertNotNull(handle);
+    Assert.assertNotNull(handle2);
 
     // get myvar session params on handle2
     try {
@@ -342,7 +347,7 @@ public class TestSessionResource extends LensJerseyTest {
 
   @Test(dataProvider = "mediaTypeData")
   public void testServerMustRestartOnManualDeletionOfAddedResources(MediaType mt)
-    throws IOException, LenServerTestException {
+    throws IOException, LensException, LenServerTestException {
 
     /* Begin: Setup */
 
@@ -361,6 +366,8 @@ public class TestSessionResource extends LensJerseyTest {
 
     /* Verification Steps: server should restart without exceptions */
     restartLensServer();
+    HiveSessionService service = LensServices.get().getService(SessionService.NAME);
+    service.closeSession(sessionHandle);
   }
 
   private LensSessionHandle openSession(final String userName, final String passwd, final LensConf conf, MediaType mt) {
@@ -371,7 +378,7 @@ public class TestSessionResource extends LensJerseyTest {
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("username").build(), userName));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("password").build(), passwd));
     mp.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionconf").fileName("sessionconf").build(),
-      conf, mt));
+        conf, mt));
 
     return target.request(mt).post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE),
       LensSessionHandle.class);
@@ -422,7 +429,8 @@ public class TestSessionResource extends LensJerseyTest {
     HiveSessionService service = LensServices.get().getService(SessionService.NAME);
     LensSessionImpl session = service.getSession(handle);
     Assert.assertEquals(session.getCurrentDatabase(), testDbName, "Expected current DB to be set to " + testDbName);
-
+    APIResult result = target.queryParam("sessionid", handle).request().delete(APIResult.class);
+    Assert.assertEquals(result.getStatus(), APIResult.Status.SUCCEEDED);
 
     // TEST 2 - Try set database with invalid db name
     final String invalidDB = testDbName + "_invalid_db";
@@ -433,7 +441,6 @@ public class TestSessionResource extends LensJerseyTest {
     form2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("database").build(), invalidDB));
     form2.bodyPart(new FormDataBodyPart(FormDataContentDisposition.name("sessionconf").fileName("sessionconf").build(),
       new LensConf(), mt));
-
     try {
       final LensSessionHandle handle2 = target.request(mt).post(Entity.entity(form2,
           MediaType.MULTIPART_FORM_DATA_TYPE), LensSessionHandle.class);
@@ -486,6 +493,74 @@ public class TestSessionResource extends LensJerseyTest {
     return mp;
   }
 
+  @Test
+  public void testMaxSessionsPerUser() throws Exception {
+    HiveSessionService sessionService = LensServices.get().getService(SessionService.NAME);
+    HiveConf conf = LensServerConf.getHiveConf();
+    Integer maxSessionsLimitPerUser = conf.getInt(LensConfConstants.MAX_SESSIONS_PER_USER,
+      LensConfConstants.DEFAULT_MAX_SESSIONS_PER_USER);
+    List<LensSessionHandle> sessions = new ArrayList<>();
+    try {
+      for (int i = 0; i < maxSessionsLimitPerUser; i++) {
+        LensSessionHandle sessionHandle = sessionService.openSession("test@localhost", "test",
+          new HashMap<String, String>());
+        sessions.add(sessionHandle);
+        Assert.assertNotNull(sessionHandle);
+      }
+      try {
+        sessionService.openSession("test@localhost", "test", new HashMap<String, String>());
+        Assert.fail("Session should not be created as session limit is already reached");
+      } catch (LensException le) {
+        // Exception expected as max session limit is reached for user
+        Assert.assertEquals(le.getErrorCode(), 2004);
+      }
+      // User should be able to open a new session by closing the one of the existing opened sessions
+      sessionService.closeSession(sessions.remove(0));
+      LensSessionHandle sessionHandle = sessionService.openSession("test@localhost", "test",
+        new HashMap<String, String>());
+      sessions.add(sessionHandle);
+      Assert.assertNotNull(sessionHandle);
+    } finally {
+      for (LensSessionHandle sessionHandle : sessions) {
+        sessionService.closeSession(sessionHandle);
+      }
+    }
+  }
+
+  @Test(dataProvider = "mediaTypeData")
+  public void testSessionLimit(MediaType mt) {
+    HiveConf conf = LensServerConf.getHiveConf();
+    Integer maxSessionsLimitPerUser = conf.getInt(LensConfConstants.MAX_SESSIONS_PER_USER,
+        LensConfConstants.DEFAULT_MAX_SESSIONS_PER_USER);
+
+    List<LensSessionHandle> sessionHandleList = new ArrayList<>();
+    try {
+      for (int i = 0; i < maxSessionsLimitPerUser; i++) {
+        final LensSessionHandle handle = RestAPITestUtil.openSession(target(), "test", "test", mt);
+        Assert.assertNotNull(handle);
+        sessionHandleList.add(handle);
+      }
+      try {
+        RestAPITestUtil.openSession(target(), "test", "test", mt);
+
+        Assert.fail("Should not open a new session for user: 'test' as user has already "
+            + maxSessionsLimitPerUser + "active sessions");
+      } catch (ClientErrorException e) {
+        Assert.assertEquals(e.getResponse().getStatus(), 429);
+      }
+      // User should be able to open a new session by closing the one of the existing opened sessions
+      RestAPITestUtil.closeSession(target(), sessionHandleList.remove(0), mt);
+
+      LensSessionHandle lensSessionHandle = RestAPITestUtil.openSession(target(), "test", "test", mt);
+      Assert.assertNotNull(lensSessionHandle);
+      sessionHandleList.add(lensSessionHandle);
+    } finally {
+      for (LensSessionHandle sessionHandle : sessionHandleList) {
+        RestAPITestUtil.closeSession(target(), sessionHandle, mt);
+      }
+    }
+  }
+
   @Test(dataProvider = "mediaTypeData")
   public void testSessionEvents(MediaType mt) {
     final WebTarget target = target().path("session");

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/java/org/apache/lens/server/ui/TestSessionUIResource.java
----------------------------------------------------------------------
diff --git a/lens-server/src/test/java/org/apache/lens/server/ui/TestSessionUIResource.java b/lens-server/src/test/java/org/apache/lens/server/ui/TestSessionUIResource.java
index be87e65..aa28747 100644
--- a/lens-server/src/test/java/org/apache/lens/server/ui/TestSessionUIResource.java
+++ b/lens-server/src/test/java/org/apache/lens/server/ui/TestSessionUIResource.java
@@ -27,10 +27,12 @@ import javax.ws.rs.core.Response;
 import org.apache.lens.api.LensConf;
 import org.apache.lens.api.LensSessionHandle;
 import org.apache.lens.server.LensJerseyTest;
+import org.apache.lens.server.api.error.LensException;
 
 import org.glassfish.jersey.media.multipart.FormDataBodyPart;
 import org.glassfish.jersey.media.multipart.FormDataContentDisposition;
 import org.glassfish.jersey.media.multipart.FormDataMultiPart;
+
 import org.testng.Assert;
 import org.testng.annotations.AfterTest;
 import org.testng.annotations.BeforeTest;
@@ -86,9 +88,9 @@ public class TestSessionUIResource extends LensJerseyTest {
    * Test ui session
    */
   @Test
-  public void testUISession() {
+  public void testUISession() throws LensException {
     final WebTarget target = target().path("uisession");
-    FormDataMultiPart mp = getMultiFormData("foo", "bar");
+    FormDataMultiPart mp = getMultiFormData("UITest", "UITest");
 
     LensSessionHandle lensSessionHandle = target.request().post(
         Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE), LensSessionHandle.class);
@@ -101,7 +103,7 @@ public class TestSessionUIResource extends LensJerseyTest {
   @Test
   public void testJsonResponsesFromServer() {
     final WebTarget target = target().path("uisession");
-    FormDataMultiPart mp = getMultiFormData("foo", "bar");
+    FormDataMultiPart mp = getMultiFormData("UITest", "UITest");
 
     Response response = target.request().accept(MediaType.APPLICATION_JSON).
         post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE));
@@ -112,12 +114,11 @@ public class TestSessionUIResource extends LensJerseyTest {
   @Test
   public void testXMLResponsesFromServer() {
     final WebTarget target = target().path("uisession");
-    FormDataMultiPart mp = getMultiFormData("foo", "bar");
+    FormDataMultiPart mp = getMultiFormData("UITest", "UITest");
 
     Response response = target.request().accept(MediaType.APPLICATION_XML).
         post(Entity.entity(mp, MediaType.MULTIPART_FORM_DATA_TYPE));
     Assert.assertEquals(response.getStatus(), 200);
     Assert.assertEquals(response.getMediaType().toString(), "application/xml");
   }
-
 }

http://git-wip-us.apache.org/repos/asf/lens/blob/0ba17ef5/lens-server/src/test/resources/lens-site.xml
----------------------------------------------------------------------
diff --git a/lens-server/src/test/resources/lens-site.xml b/lens-server/src/test/resources/lens-site.xml
index c3187a8..b5b3220 100644
--- a/lens-server/src/test/resources/lens-site.xml
+++ b/lens-server/src/test/resources/lens-site.xml
@@ -187,4 +187,9 @@
     <name>lens.server.estimate.timeout.millis</name>
     <value>120000</value>
   </property>
+  <property>
+    <name>lens.server.max.sessions.per.user</name>
+    <value>20</value>
+    <description>Number of sessions can be allowed for each user.</description>
+  </property>
 </configuration>