You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@knox.apache.org by GitBox <gi...@apache.org> on 2020/07/14 12:47:50 UTC

[GitHub] [knox] smolnar82 opened a new pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

smolnar82 opened a new pull request #361:
URL: https://github.com/apache/knox/pull/361


   ## What changes were proposed in this pull request?
   
   The main purpose of this PR is to add a new token state service implementation the uses Zookeeper as token storage.
   Also, the following enhancements were made:
   - introduced a service cache in `org.apache.knox.gateway.services.factory.AbstractServiceFactory` to avoid creating/initiating the same service by type/implementation more than once
   - `ZookeeperRemoteAliasService` can be built w/o local alias service
   
   ## How was this patch tested?
   
   Updated/ran JUnit tests and executed the following manual test steps:
   
   1. Enabled token state service in sandbox:
   ```
         <param>
             <name>knox.token.exp.server-managed</name>
             <value>true</value>
         </param>
   ```
   2. Configured Knox to use the new ZK Token State Service in `gateway-site.xml`:
   ```
       <property>
           <name>gateway.service.tokenstate.impl</name>
           <value>org.apache.knox.gateway.services.token.impl.ZookeeperTokenStateService</value>
       </property>
       <property>
           <name>gateway.service.tokenstate.useLocalAliasService</name>
           <value>[true|false]</value>
       </property>
       <property>
           <name>gateway.remote.config.registry.zookeeper-client</name>
           <value>type=ZooKeeper;address=$ZK_HOST:2181</value>
           <description>ZooKeeper configuration registry client details.</description>
       </property>
       <property>
           <name>gateway.remote.config.monitor.client</name>
           <value>zookeeper-client</value>
           <description>Remote configuration monitor client name.</description>
       </property>
   ```
   3. Started Knox
   4. Acquired a Knox delegation token:
   ```
   $ curl -ivku guest:guest-password https://localhost:8443/gateway/sandbox/knoxtoken/api/v1/token
   ...
   * Connection #0 to host localhost left intact
   {"access_token":"eyJhbGciOiJSUzI1NiJ9.eyJzdWIiOiJndWVzdCIsImF1...,"target_url":"https://localhost:8443/gateway/tokenbased",...,"token_type":"Bearer","expires_in":1594764588683}
   
   gateway.log:
   2020-07-14 14:09:48,681 INFO  service.knoxtoken (TokenResource.java:init(169)) - Server management of token state is enabled for the "sandbox" topology.
   2020-07-14 14:09:48,682 WARN  service.knoxtoken (TokenResource.java:init(199)) - There are no token renewers white-listed in the "sandbox" topology.
   2020-07-14 14:09:48,690 INFO  service.knoxtoken (TokenResource.java:getAuthenticationToken(400)) - Knox Token service (sandbox) issued token eyJhbG...WYtr9Q (483b5976-f5ca-4761-b2d3-65041aa40d09)
   ```
   5. Confirmed that the token became persisted in Zookeeper:
   ```
   $ zookeeper-client -server $ZK_HOST:2181
   Connecting to $ZK_HOST:2181
   Welcome to ZooKeeper!
   JLine support is enabled
   
   WATCHER::
   
   WatchedEvent state:SyncConnected type:None path:null
   
   [zk: $ZK_HOST:2181(CONNECTED) 0] ls /knox/security/topology/__gateway
   [483b5976-f5ca-4761-b2d3-65041aa40d09, 483b5976-f5ca-4761-b2d3-65041aa40d09--max]
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454998600



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {
+          // no known service implementation created, try to create the custom one
+          try {
+            service = Service.class.cast(Class.forName(implementation).newInstance());

Review comment:
       Changed anyway.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454998712



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {
+          // no known service implementation created, try to create the custom one
+          try {
+            service = Service.class.cast(Class.forName(implementation).newInstance());
+            logServiceUsage(implementation, serviceType);
+          } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+            throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);

Review comment:
       Fixed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454396523



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -37,7 +41,8 @@
 
   private static final GatewayMessages LOG = MessagesFactory.get(GatewayMessages.class);
   private static final String IMPLEMENTATION_PARAM_NAME = "impl";
-  private static final String EMPTY_DEFAULT_IMPLEMENTATION = "";
+  protected static final String EMPTY_DEFAULT_IMPLEMENTATION = "";
+  private final Map<ServiceCacheKey, Service> serviceCache = new HashMap<>(); //to avoid creating the same service with the same implementation more than once

Review comment:
       Removed that code since.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454391965



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/security/impl/ZookeeperRemoteAliasService.java
##########
@@ -190,13 +190,18 @@ private static void checkPathsExist(final RemoteConfigurationRegistryClient remo
      *
      * @param clusterName
      *            cluster name
-     * @return List of all the aliases
+     * @return List of all the aliases (an empty list, in case there is no alias for the given cluster)
      */
     @Override
     public List<String> getAliasesForCluster(final String clusterName) throws AliasServiceException {
-        final List<String> localAliases = localAliasService.getAliasesForCluster(clusterName);
+        final List<String> localAliases = localAliasService == null ? Collections.emptyList() : localAliasService.getAliasesForCluster(clusterName);
         if (localAliases == null || localAliases.isEmpty()) {
-          return remoteClient == null ? new ArrayList<>() : remoteClient.listChildEntries(buildClusterEntryName(clusterName));
+          if (remoteClient != null) {
+            final List<String> remoteAliases = remoteClient.listChildEntries(buildClusterEntryName(clusterName));
+            return remoteAliases == null ? new ArrayList<>() : remoteAliases;
+          } else {
+            return new ArrayList<>();

Review comment:
       I've no idea if it should be mutable and I did not want to touch that part -> this is why I use the same way to return in the other case too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
pzampino commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454401688



##########
File path: gateway-server/src/test/java/org/apache/knox/gateway/services/factory/ServiceFactoryTest.java
##########
@@ -52,7 +52,7 @@
 
   protected final GatewayServices gatewayServices = EasyMock.createNiceMock(GatewayServices.class);
   protected final GatewayConfig gatewayConfig = EasyMock.createNiceMock(GatewayConfig.class);
-  protected final Map<String, String> options = Collections.emptyMap();
+  protected final Map<String, String> options = new HashMap<>();

Review comment:
       Does the mutability of the map matter for a _test_?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454417057



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/TokenStateServiceFactory.java
##########
@@ -61,6 +64,7 @@ protected ServiceType getServiceType() {
 
   @Override
   protected Collection<String> getKnownImplementations() {
-    return unmodifiableList(asList(DefaultTokenStateService.class.getName(), AliasBasedTokenStateService.class.getName(), JournalBasedTokenStateService.class.getName()));
+    return unmodifiableList(asList(DefaultTokenStateService.class.getName(), AliasBasedTokenStateService.class.getName(), JournalBasedTokenStateService.class.getName(),

Review comment:
       We may implement this enhancement in a different JIRA, but I think it's out-of-scope of this one.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 merged pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 merged pull request #361:
URL: https://github.com/apache/knox/pull/361


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454390468



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AliasServiceFactory.java
##########
@@ -54,7 +57,8 @@ protected Service createService(GatewayServices gatewayServices, ServiceType ser
       } else if (matchesImplementation(implementation, RemoteAliasService.class)) {
         service = new RemoteAliasService(defaultAliasService, getMasterService(gatewayServices));
       } else if (matchesImplementation(implementation, ZookeeperRemoteAliasService.class)) {
-        service = new ZookeeperRemoteAliasServiceProvider().newInstance(defaultAliasService, getMasterService(gatewayServices));
+        final boolean useLocalAliasService = options == null ? true : Boolean.valueOf(options.getOrDefault(USE_LOCAL_ALIAS_IN_ZK_IMPL_PARAM_NAME, "true"));
+        service = new ZookeeperRemoteAliasServiceProvider().newInstance(useLocalAliasService ? defaultAliasService : null, getMasterService(gatewayServices));

Review comment:
       Nope. It means if you set the implementation to ZK in gateway-site.xml, then tokens will be stored in ZK. The default is `DefaultAliasService`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino commented on pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
pzampino commented on pull request #361:
URL: https://github.com/apache/knox/pull/361#issuecomment-658221309


   > I have few concerns
   > 
   > 3. ZK is not not build to store data in this case we are storing sensitive data. In case of loss of quorum TSS will be significantly impacted (unless we store the tokens locally and then sync to ZK). Also, if we push a whole bunch of tokens in ZK we might impact other services negatively.
   
   Since this implementation is based on the AliasBasedTokenStateService, it also employs a local cache and the token state journaling facility to mitigate potential token state loss. Further, I think the ZookeeperRemoteAliasService (used by this implementation) keeps the local keystore in sync, correct?
   
   > 4. I foresee performance issues in case token size increases.
   
   There is certainly performance testing yet to be done, at least to understand the limitations of this implementation.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454389474



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -37,7 +41,8 @@
 
   private static final GatewayMessages LOG = MessagesFactory.get(GatewayMessages.class);
   private static final String IMPLEMENTATION_PARAM_NAME = "impl";
-  private static final String EMPTY_DEFAULT_IMPLEMENTATION = "";
+  protected static final String EMPTY_DEFAULT_IMPLEMENTATION = "";
+  private final Map<ServiceCacheKey, Service> serviceCache = new HashMap<>(); //to avoid creating the same service with the same implementation more than once

Review comment:
       Removed it from the latest patchset.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454417260



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/ZookeeperTokenStateService.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.knox.gateway.services.token.impl;
+
+import static org.apache.knox.gateway.services.ServiceType.ALIAS_SERVICE;
+
+import java.util.Map;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.services.GatewayServices;
+import org.apache.knox.gateway.services.ServiceLifecycleException;
+import org.apache.knox.gateway.services.factory.AliasServiceFactory;
+import org.apache.knox.gateway.services.security.impl.ZookeeperRemoteAliasService;
+
+/**
+ * A Zookeeper Token State Service is actually an Alias based TSS where the 'alias service' happens to be the 'zookeeper' implementation.
+ * This means the only important thing that should be overridden here is the init method where the underlying alias service is configured
+ * properly.
+ */
+public class ZookeeperTokenStateService extends AliasBasedTokenStateService {

Review comment:
       Yes, this is a good idea.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
pzampino commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454404824



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {
+          // no known service implementation created, try to create the custom one
+          try {
+            service = Service.class.cast(Class.forName(implementation).newInstance());

Review comment:
       I don't believe there is any functional difference between the two. The _cast_ method didn't exist until generics were added to Java.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454392757



##########
File path: gateway-server/src/test/java/org/apache/knox/gateway/services/factory/ServiceFactoryTest.java
##########
@@ -52,7 +52,7 @@
 
   protected final GatewayServices gatewayServices = EasyMock.createNiceMock(GatewayServices.class);
   protected final GatewayConfig gatewayConfig = EasyMock.createNiceMock(GatewayConfig.class);
-  protected final Map<String, String> options = Collections.emptyMap();
+  protected final Map<String, String> options = new HashMap<>();

Review comment:
       We _might_ need. In one of our tests, I modified `options` (removed that since then) and the test failed since the map was immutable.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454998470



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/ZookeeperTokenStateService.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.knox.gateway.services.token.impl;
+
+import static org.apache.knox.gateway.services.ServiceType.ALIAS_SERVICE;
+
+import java.util.Map;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.services.GatewayServices;
+import org.apache.knox.gateway.services.ServiceLifecycleException;
+import org.apache.knox.gateway.services.factory.AliasServiceFactory;
+import org.apache.knox.gateway.services.security.impl.ZookeeperRemoteAliasService;
+
+/**
+ * A Zookeeper Token State Service is actually an Alias based TSS where the 'alias service' happens to be the 'zookeeper' implementation.
+ * This means the only important thing that should be overridden here is the init method where the underlying alias service is configured
+ * properly.
+ */
+public class ZookeeperTokenStateService extends AliasBasedTokenStateService {

Review comment:
       Added a brand new test class that actually starts-up a ZK cluster and the new ZK Token State Service stores the token aliases there: `org.apache.knox.gateway.services.token.impl.ZookeeperTokenStateServiceTest`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454388111



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {
+          // no known service implementation created, try to create the custom one
+          try {
+            service = Service.class.cast(Class.forName(implementation).newInstance());
+            logServiceUsage(implementation, serviceType);
+          } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+            throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);

Review comment:
       Because it is an ERRRRRRRRROR! :) I'll fix it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] moresandeep commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
moresandeep commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454357167



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -37,7 +41,8 @@
 
   private static final GatewayMessages LOG = MessagesFactory.get(GatewayMessages.class);
   private static final String IMPLEMENTATION_PARAM_NAME = "impl";
-  private static final String EMPTY_DEFAULT_IMPLEMENTATION = "";
+  protected static final String EMPTY_DEFAULT_IMPLEMENTATION = "";
+  private final Map<ServiceCacheKey, Service> serviceCache = new HashMap<>(); //to avoid creating the same service with the same implementation more than once

Review comment:
       This code runs during init right then what would be the scenario where there would be multiple implementation of same service?

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {
+          // no known service implementation created, try to create the custom one
+          try {
+            service = Service.class.cast(Class.forName(implementation).newInstance());

Review comment:
       This looks awkward, this can be changed to `(Service) Class.forName(implementation).newInstance();`

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {

Review comment:
       I think we should have a fallback option in case `implementation` is missing - a meaningful default perhaps?

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -37,7 +41,8 @@
 
   private static final GatewayMessages LOG = MessagesFactory.get(GatewayMessages.class);
   private static final String IMPLEMENTATION_PARAM_NAME = "impl";
-  private static final String EMPTY_DEFAULT_IMPLEMENTATION = "";
+  protected static final String EMPTY_DEFAULT_IMPLEMENTATION = "";
+  private final Map<ServiceCacheKey, Service> serviceCache = new HashMap<>(); //to avoid creating the same service with the same implementation more than once

Review comment:
       Might be better to use `ConcurrentHashMap` just because we are using it for Caching.

##########
File path: gateway-server/src/test/java/org/apache/knox/gateway/services/factory/ServiceFactoryTest.java
##########
@@ -52,7 +52,7 @@
 
   protected final GatewayServices gatewayServices = EasyMock.createNiceMock(GatewayServices.class);
   protected final GatewayConfig gatewayConfig = EasyMock.createNiceMock(GatewayConfig.class);
-  protected final Map<String, String> options = Collections.emptyMap();
+  protected final Map<String, String> options = new HashMap<>();

Review comment:
       Any reason why we need a mutable map?

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/security/impl/ZookeeperRemoteAliasService.java
##########
@@ -190,13 +190,18 @@ private static void checkPathsExist(final RemoteConfigurationRegistryClient remo
      *
      * @param clusterName
      *            cluster name
-     * @return List of all the aliases
+     * @return List of all the aliases (an empty list, in case there is no alias for the given cluster)
      */
     @Override
     public List<String> getAliasesForCluster(final String clusterName) throws AliasServiceException {
-        final List<String> localAliases = localAliasService.getAliasesForCluster(clusterName);
+        final List<String> localAliases = localAliasService == null ? Collections.emptyList() : localAliasService.getAliasesForCluster(clusterName);
         if (localAliases == null || localAliases.isEmpty()) {
-          return remoteClient == null ? new ArrayList<>() : remoteClient.listChildEntries(buildClusterEntryName(clusterName));
+          if (remoteClient != null) {
+            final List<String> remoteAliases = remoteClient.listChildEntries(buildClusterEntryName(clusterName));
+            return remoteAliases == null ? new ArrayList<>() : remoteAliases;

Review comment:
       `Collections.emptyList() ` might be better here too like you did above.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {
+          // no known service implementation created, try to create the custom one
+          try {
+            service = Service.class.cast(Class.forName(implementation).newInstance());
+            logServiceUsage(implementation, serviceType);
+          } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
+            throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);

Review comment:
       Typo in Error :)

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AliasServiceFactory.java
##########
@@ -54,7 +57,8 @@ protected Service createService(GatewayServices gatewayServices, ServiceType ser
       } else if (matchesImplementation(implementation, RemoteAliasService.class)) {
         service = new RemoteAliasService(defaultAliasService, getMasterService(gatewayServices));
       } else if (matchesImplementation(implementation, ZookeeperRemoteAliasService.class)) {
-        service = new ZookeeperRemoteAliasServiceProvider().newInstance(defaultAliasService, getMasterService(gatewayServices));
+        final boolean useLocalAliasService = options == null ? true : Boolean.valueOf(options.getOrDefault(USE_LOCAL_ALIAS_IN_ZK_IMPL_PARAM_NAME, "true"));
+        service = new ZookeeperRemoteAliasServiceProvider().newInstance(useLocalAliasService ? defaultAliasService : null, getMasterService(gatewayServices));

Review comment:
       Does this mean the default behavior is to store tokens in ZK?

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/security/impl/ZookeeperRemoteAliasService.java
##########
@@ -190,13 +190,18 @@ private static void checkPathsExist(final RemoteConfigurationRegistryClient remo
      *
      * @param clusterName
      *            cluster name
-     * @return List of all the aliases
+     * @return List of all the aliases (an empty list, in case there is no alias for the given cluster)
      */
     @Override
     public List<String> getAliasesForCluster(final String clusterName) throws AliasServiceException {
-        final List<String> localAliases = localAliasService.getAliasesForCluster(clusterName);
+        final List<String> localAliases = localAliasService == null ? Collections.emptyList() : localAliasService.getAliasesForCluster(clusterName);
         if (localAliases == null || localAliases.isEmpty()) {
-          return remoteClient == null ? new ArrayList<>() : remoteClient.listChildEntries(buildClusterEntryName(clusterName));
+          if (remoteClient != null) {
+            final List<String> remoteAliases = remoteClient.listChildEntries(buildClusterEntryName(clusterName));
+            return remoteAliases == null ? new ArrayList<>() : remoteAliases;
+          } else {
+            return new ArrayList<>();

Review comment:
       `Collections.emptyList() ` might be better here too like you did above. Unless this is mutable list.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] pzampino commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
pzampino commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454400403



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/token/impl/ZookeeperTokenStateService.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.knox.gateway.services.token.impl;
+
+import static org.apache.knox.gateway.services.ServiceType.ALIAS_SERVICE;
+
+import java.util.Map;
+
+import org.apache.knox.gateway.config.GatewayConfig;
+import org.apache.knox.gateway.services.GatewayServices;
+import org.apache.knox.gateway.services.ServiceLifecycleException;
+import org.apache.knox.gateway.services.factory.AliasServiceFactory;
+import org.apache.knox.gateway.services.security.impl.ZookeeperRemoteAliasService;
+
+/**
+ * A Zookeeper Token State Service is actually an Alias based TSS where the 'alias service' happens to be the 'zookeeper' implementation.
+ * This means the only important thing that should be overridden here is the init method where the underlying alias service is configured
+ * properly.
+ */
+public class ZookeeperTokenStateService extends AliasBasedTokenStateService {

Review comment:
       I know it's a bit more work, but can you extend AliasBasedTokenStateServiceTest to exercise this new implementation?

##########
File path: gateway-server/src/test/java/org/apache/knox/gateway/services/factory/TokenStateServiceFactoryTest.java
##########
@@ -43,22 +44,31 @@ public void testBasics() throws Exception {
 
   @Test
   public void shouldReturnDefaultAliasService() throws Exception {

Review comment:
       Method name indicates an alias service test, but the implementation is a TokenStateService test.

##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/TokenStateServiceFactory.java
##########
@@ -61,6 +64,7 @@ protected ServiceType getServiceType() {
 
   @Override
   protected Collection<String> getKnownImplementations() {
-    return unmodifiableList(asList(DefaultTokenStateService.class.getName(), AliasBasedTokenStateService.class.getName(), JournalBasedTokenStateService.class.getName()));
+    return unmodifiableList(asList(DefaultTokenStateService.class.getName(), AliasBasedTokenStateService.class.getName(), JournalBasedTokenStateService.class.getName(),

Review comment:
       This is not necessarily directly related to this PR, but why isn't the ServiceLoader mechanism used for identifying service implementations available on the classpath?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454416041



##########
File path: gateway-server/src/test/java/org/apache/knox/gateway/services/factory/ServiceFactoryTest.java
##########
@@ -52,7 +52,7 @@
 
   protected final GatewayServices gatewayServices = EasyMock.createNiceMock(GatewayServices.class);
   protected final GatewayConfig gatewayConfig = EasyMock.createNiceMock(GatewayConfig.class);
-  protected final Map<String, String> options = Collections.emptyMap();
+  protected final Map<String, String> options = new HashMap<>();

Review comment:
       Yes. Since this is a base class for other tests this `options` instance is used within test classes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454387621



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {
+          // no known service implementation created, try to create the custom one
+          try {
+            service = Service.class.cast(Class.forName(implementation).newInstance());

Review comment:
       I believe this depends on personal tasting. I like the one I added before too.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454390890



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/security/impl/ZookeeperRemoteAliasService.java
##########
@@ -190,13 +190,18 @@ private static void checkPathsExist(final RemoteConfigurationRegistryClient remo
      *
      * @param clusterName
      *            cluster name
-     * @return List of all the aliases
+     * @return List of all the aliases (an empty list, in case there is no alias for the given cluster)
      */
     @Override
     public List<String> getAliasesForCluster(final String clusterName) throws AliasServiceException {
-        final List<String> localAliases = localAliasService.getAliasesForCluster(clusterName);
+        final List<String> localAliases = localAliasService == null ? Collections.emptyList() : localAliasService.getAliasesForCluster(clusterName);
         if (localAliases == null || localAliases.isEmpty()) {
-          return remoteClient == null ? new ArrayList<>() : remoteClient.listChildEntries(buildClusterEntryName(clusterName));
+          if (remoteClient != null) {
+            final List<String> remoteAliases = remoteClient.listChildEntries(buildClusterEntryName(clusterName));
+            return remoteAliases == null ? new ArrayList<>() : remoteAliases;

Review comment:
       I just wanted to be consistent with the `null` case.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [knox] smolnar82 commented on a change in pull request #361: KNOX-2399 - Implemented ZookeeperTokenStateService

Posted by GitBox <gi...@apache.org>.
smolnar82 commented on a change in pull request #361:
URL: https://github.com/apache/knox/pull/361#discussion_r454396302



##########
File path: gateway-server/src/main/java/org/apache/knox/gateway/services/factory/AbstractServiceFactory.java
##########
@@ -47,22 +52,38 @@ public Service create(GatewayServices gatewayServices, ServiceType serviceType,
   @Override
   public Service create(GatewayServices gatewayServices, ServiceType serviceType, GatewayConfig gatewayConfig, Map<String, String> options, String implementation)
       throws ServiceLifecycleException {
-    Service service = null;
-    if (getServiceType() == serviceType) {
-      service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
-      if (service == null && StringUtils.isNotBlank(implementation)) {
-        // no known service implementation created, try to create the custom one
-        try {
-          service = Service.class.cast(Class.forName(implementation).newInstance());
-          logServiceUsage(implementation, serviceType);
-        } catch (InstantiationException | IllegalAccessException | ClassNotFoundException e) {
-          throw new ServiceLifecycleException("Errror while instantiating " + serviceType.getShortName() + " service implementation " + implementation, e);
+    Service service = fetchFromCache(serviceType, implementation);
+    if (service == null) {
+      if (getServiceType() == serviceType) {
+        service = createService(gatewayServices, serviceType, gatewayConfig, options, implementation);
+        if (service == null && StringUtils.isNotBlank(implementation)) {

Review comment:
       We **do** have a fallback. There are 2 types of service factories:
   
   1. there is only one known implementation - in this case, that known implementation will be created if it's not overwritten in `gatewy-site.xml`
   2. there are more than one known implementations - in this case, one of the known implementations (typically the first one in the `if-else` branches) will be the default one (i.e. none is defined in `gateway-site.xml`)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org