You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gobblin.apache.org by GitBox <gi...@apache.org> on 2021/03/03 21:09:48 UTC

[GitHub] [gobblin] aplex commented on a change in pull request #3238: [GOBBLIN-1402] Allow flow's requester list/owner to be updated

aplex commented on a change in pull request #3238:
URL: https://github.com/apache/gobblin/pull/3238#discussion_r586770996



##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2Resource.java
##########
@@ -208,18 +217,59 @@ private Properties getHeaders() {
     return headerProperties;
   }
 
+  /**
+   * Check that this update or delete operation is allowed, throw a {@link FlowConfigLoggedException} if not.
+   */
+  public void checkUpdateDeleteAllowed(FlowConfig originalFlowConfig, FlowConfig updatedFlowConfig) {
+    List<ServiceRequester> requesterList = this.requesterService.findRequesters(this);
+    if (updatedFlowConfig != null) {
+      checkPropertyUpdatesAllowed(requesterList, updatedFlowConfig);
+    }
+    checkRequester(originalFlowConfig, requesterList);
+  }
+
+  /**
+   * Check that the properties being updated are allowed to be updated. This includes:
+   * 1. Checking that the requester is part of the owningGroup if it is being modified
+   * 2. Checking if the {@link RequesterService#REQUESTER_LIST} is being modified, and only allow it if a user is changing
+   *    it to themselves.
+   */
+  public void checkPropertyUpdatesAllowed(List<ServiceRequester> requesterList, FlowConfig updatedFlowConfig) {
+    if (requesterList == null || this.requesterService.isRequesterWhitelisted(requesterList)) {
+      return;
+    }
+
+    // Check that requester is part of owning group if owning group is being updated
+    if (updatedFlowConfig.hasOwningGroup() && !this.groupOwnershipService.isMemberOfGroup(requesterList, updatedFlowConfig.getOwningGroup())) {
+      throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, "Requester not part of owning group specified");
+    }
+
+    if (updatedFlowConfig.hasProperties() && updatedFlowConfig.getProperties().containsKey(RequesterService.REQUESTER_LIST)) {
+      List<ServiceRequester> updatedRequesterList;
+      try {
+        updatedRequesterList = RequesterService.deserialize(updatedFlowConfig.getProperties().get(RequesterService.REQUESTER_LIST));
+      } catch (Exception e) {
+        throw new FlowConfigLoggedException(HttpStatus.S_400_BAD_REQUEST, RequesterService.REQUESTER_LIST + " property was "
+            + "provided but could not be deserialized", e);
+      }
+
+      if (!updatedRequesterList.equals(requesterList)) {
+        throw new FlowConfigLoggedException(HttpStatus.S_401_UNAUTHORIZED, RequesterService.REQUESTER_LIST + " property may "
+            + "only be updated to yourself. Requesting user: " + requesterList + ", updated requester: " + updatedRequesterList);
+      }
+    }
+  }
+
   /**
    * Check that all {@link ServiceRequester}s in this request are contained within the original service requester list
    * or is part of the original requester's owning group when the flow was submitted. If they are not, throw a {@link FlowConfigLoggedException} with {@link HttpStatus#S_401_UNAUTHORIZED}.
    * If there is a failure when deserializing the original requester list, throw a {@link FlowConfigLoggedException} with
    * {@link HttpStatus#S_400_BAD_REQUEST}.
-   * @param requesterService the {@link RequesterService} used to verify the requester
    * @param originalFlowConfig original flow config to find original requester
    * @param requesterList list of requesters for this request
    */
-  public void checkRequester(
-      RequesterService requesterService, FlowConfig originalFlowConfig, List<ServiceRequester> requesterList) {
-    if (requesterList == null) {
+  public void checkRequester(FlowConfig originalFlowConfig, List<ServiceRequester> requesterList) {
+    if (requesterList == null || this.requesterService.isRequesterWhitelisted(requesterList)) {

Review comment:
       When can requesterList be null? Looks like if it is not set, we'll allow all operations.

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java
##########
@@ -301,6 +302,42 @@ public void testLocalGroupOwnershipUpdates() throws Exception {
     _client.deleteFlowConfig(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_7));
   }
 
+
+  @Test (expectedExceptions = RestLiResponseException.class)
+  public void testGroupUpdateRejected() throws Exception {
+   ServiceRequester testRequester = new ServiceRequester("testName", "USER_PRINCIPAL", "testFrom");
+   _requesterService.setRequester(testRequester);
+   Map<String, String> flowProperties = Maps.newHashMap();
+
+   FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_8))
+       .setTemplateUris(TEST_TEMPLATE_URI)
+       .setProperties(new StringMap(flowProperties))
+       .setOwningGroup("testGroup");
+
+   _client.createFlowConfig(flowConfig);
+
+   flowConfig.setOwningGroup("testGroup2");
+   _client.updateFlowConfig(flowConfig);
+  }
+
+  @Test (expectedExceptions = RestLiResponseException.class)
+  public void testRequesterUpdateRejected() throws Exception {

Review comment:
       Do we have a test that will check that requester can be updated?

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java
##########
@@ -301,6 +302,42 @@ public void testLocalGroupOwnershipUpdates() throws Exception {
     _client.deleteFlowConfig(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_7));
   }
 
+
+  @Test (expectedExceptions = RestLiResponseException.class)
+  public void testGroupUpdateRejected() throws Exception {
+   ServiceRequester testRequester = new ServiceRequester("testName", "USER_PRINCIPAL", "testFrom");
+   _requesterService.setRequester(testRequester);
+   Map<String, String> flowProperties = Maps.newHashMap();
+
+   FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_8))
+       .setTemplateUris(TEST_TEMPLATE_URI)
+       .setProperties(new StringMap(flowProperties))
+       .setOwningGroup("testGroup");
+
+   _client.createFlowConfig(flowConfig);
+
+   flowConfig.setOwningGroup("testGroup2");
+   _client.updateFlowConfig(flowConfig);

Review comment:
       Why would we get exception here? From what I see, user creates a flow and then sets the owning group for the flow that he owns.

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java
##########
@@ -301,6 +302,42 @@ public void testLocalGroupOwnershipUpdates() throws Exception {
     _client.deleteFlowConfig(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_7));
   }
 
+
+  @Test (expectedExceptions = RestLiResponseException.class)
+  public void testGroupUpdateRejected() throws Exception {
+   ServiceRequester testRequester = new ServiceRequester("testName", "USER_PRINCIPAL", "testFrom");
+   _requesterService.setRequester(testRequester);
+   Map<String, String> flowProperties = Maps.newHashMap();
+
+   FlowConfig flowConfig = new FlowConfig().setId(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_8))
+       .setTemplateUris(TEST_TEMPLATE_URI)
+       .setProperties(new StringMap(flowProperties))
+       .setOwningGroup("testGroup");
+
+   _client.createFlowConfig(flowConfig);
+
+   flowConfig.setOwningGroup("testGroup2");
+   _client.updateFlowConfig(flowConfig);
+  }
+
+  @Test (expectedExceptions = RestLiResponseException.class)

Review comment:
       Can we also check that exception message or http result is the one that we expect, and it is about authorization/permissions? Currently if there is a typo in the test, and service rejects the request for any reason, the test will pass.
   
   TestNG has expectedExceptionsMessageRegExp: https://howtodoinjava.com/testng/testng-expected-exception/

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/test/java/org/apache/gobblin/service/FlowConfigV2Test.java
##########
@@ -301,6 +302,42 @@ public void testLocalGroupOwnershipUpdates() throws Exception {
     _client.deleteFlowConfig(new FlowId().setFlowGroup(TEST_GROUP_NAME).setFlowName(TEST_FLOW_NAME_7));
   }
 
+
+  @Test (expectedExceptions = RestLiResponseException.class)

Review comment:
       The design doc also mentioned that "A user that is part of the owning group can take control of the flow by updating the owner to themselves". Is this implemented/covered with tests?




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