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 2020/10/29 16:35:36 UTC

[GitHub] [incubator-gobblin] sv2000 commented on a change in pull request #3142: [GOBBLIN-1304] Adds group ownership service

sv2000 commented on a change in pull request #3142:
URL: https://github.com/apache/incubator-gobblin/pull/3142#discussion_r514363955



##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/GroupOwnershipService.java
##########
@@ -0,0 +1,27 @@
+/*
+ * 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.gobblin.service;
+
+import java.util.List;
+
+
+public abstract class GroupOwnershipService {

Review comment:
       Add javadoc for this abstract class as well as its public methods.

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LocalGroupOwnershipService.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.gobblin.service;
+
+import com.google.gson.JsonObject;
+import com.typesafe.config.Config;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import com.google.gson.JsonParser;
+
+
+public class LocalGroupOwnershipService extends GroupOwnershipService {
+  public static final String GROUP_MEMBER_LIST = ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "groupMembers.path";
+
+  JsonObject groupOwnerships;
+
+  public LocalGroupOwnershipService(Config config) {
+    Path groupOwnershipFilePath = new Path(config.getString(GROUP_MEMBER_LIST));

Review comment:
       I like the idea of having a separate data file to maintain the group ownerships and keeping its deployment separate from the service. It would be good to add the logic to reload the config on detecting a change in the file. Take a look at the PathAlterationObserver class that can help with this. But I am ok to not hold up this PR if substantial changes are needed.

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LocalGroupOwnershipService.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.gobblin.service;
+
+import com.google.gson.JsonObject;
+import com.typesafe.config.Config;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import com.google.gson.JsonParser;
+
+
+public class LocalGroupOwnershipService extends GroupOwnershipService {
+  public static final String GROUP_MEMBER_LIST = ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "groupMembers.path";
+
+  JsonObject groupOwnerships;
+
+  public LocalGroupOwnershipService(Config config) {
+    Path groupOwnershipFilePath = new Path(config.getString(GROUP_MEMBER_LIST));
+    FSDataInputStream in = null;
+    try {
+      FileSystem fs = FileSystem.get(new Configuration());
+      in = fs.open(groupOwnershipFilePath);
+      String jsonString = IOUtils.toString(in, Charset.defaultCharset());
+      JsonParser parser = new JsonParser();
+      this.groupOwnerships = parser.parse(jsonString).getAsJsonObject();
+      in.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public boolean isMemberOfGroup(List<ServiceRequester> serviceRequesters, String group) {
+    if (this.groupOwnerships.has(group)) {
+      String member = extractRequesterUser(serviceRequesters);
+      List<String> groupMembers = Arrays.asList(groupOwnerships.get(group).getAsString().split(","));
+      return groupMembers.contains(member);
+    }
+    return false;
+  }
+
+  private static String extractRequesterUser(List<ServiceRequester> requesterList) {
+    return requesterList.stream()
+        .filter(serviceRequester -> serviceRequester.getType().equals(USER_TYPE))
+        .collect(Collectors.toList())
+        .get(0)

Review comment:
       Potential NPE in case of requests made by non USER principals.

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2Resource.java
##########
@@ -200,4 +210,29 @@ private Properties getHeaders() {
     }
     return headerProperties;
   }
+
+  public void checkRequester(

Review comment:
       Add javadoc explaining the behavior of this method. 

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LocalGroupOwnershipService.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.gobblin.service;
+
+import com.google.gson.JsonObject;
+import com.typesafe.config.Config;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import com.google.gson.JsonParser;
+
+
+public class LocalGroupOwnershipService extends GroupOwnershipService {
+  public static final String GROUP_MEMBER_LIST = ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "groupMembers.path";
+
+  JsonObject groupOwnerships;
+
+  public LocalGroupOwnershipService(Config config) {
+    Path groupOwnershipFilePath = new Path(config.getString(GROUP_MEMBER_LIST));
+    FSDataInputStream in = null;
+    try {
+      FileSystem fs = FileSystem.get(new Configuration());
+      in = fs.open(groupOwnershipFilePath);
+      String jsonString = IOUtils.toString(in, Charset.defaultCharset());
+      JsonParser parser = new JsonParser();
+      this.groupOwnerships = parser.parse(jsonString).getAsJsonObject();
+      in.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public boolean isMemberOfGroup(List<ServiceRequester> serviceRequesters, String group) {
+    if (this.groupOwnerships.has(group)) {
+      String member = extractRequesterUser(serviceRequesters);
+      List<String> groupMembers = Arrays.asList(groupOwnerships.get(group).getAsString().split(","));
+      return groupMembers.contains(member);
+    }
+    return false;
+  }
+
+  private static String extractRequesterUser(List<ServiceRequester> requesterList) {
+    return requesterList.stream()
+        .filter(serviceRequester -> serviceRequester.getType().equals(USER_TYPE))

Review comment:
       We shouldn't assume every request is made by a USER principal. There could be use cases where a SERVICE principal can make a request too. 

##########
File path: gobblin-service/src/main/java/org/apache/gobblin/service/modules/core/GobblinServiceManager.java
##########
@@ -285,6 +287,13 @@ public GobblinServiceManager(String serviceName, String serviceId, Config config
     this.isRestLIServerEnabled = ConfigUtils.getBoolean(config,
         ServiceConfigKeys.GOBBLIN_SERVICE_RESTLI_SERVER_ENABLED_KEY, true);
 
+    GroupOwnershipService groupOwnershipService = GobblinConstructorUtils.invokeConstructor(

Review comment:
       Should we introduce aliases for the different group ownership service implementations and use an alias resolver to instantiate the configured group ownership service?

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LocalGroupOwnershipService.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.gobblin.service;
+
+import com.google.gson.JsonObject;
+import com.typesafe.config.Config;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import com.google.gson.JsonParser;
+
+
+public class LocalGroupOwnershipService extends GroupOwnershipService {
+  public static final String GROUP_MEMBER_LIST = ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "groupMembers.path";
+
+  JsonObject groupOwnerships;
+
+  public LocalGroupOwnershipService(Config config) {
+    Path groupOwnershipFilePath = new Path(config.getString(GROUP_MEMBER_LIST));
+    FSDataInputStream in = null;
+    try {
+      FileSystem fs = FileSystem.get(new Configuration());
+      in = fs.open(groupOwnershipFilePath);
+      String jsonString = IOUtils.toString(in, Charset.defaultCharset());
+      JsonParser parser = new JsonParser();
+      this.groupOwnerships = parser.parse(jsonString).getAsJsonObject();
+      in.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public boolean isMemberOfGroup(List<ServiceRequester> serviceRequesters, String group) {
+    if (this.groupOwnerships.has(group)) {
+      String member = extractRequesterUser(serviceRequesters);
+      List<String> groupMembers = Arrays.asList(groupOwnerships.get(group).getAsString().split(","));

Review comment:
       You can use Guava's Splitter:
   Splitter.on(',').trimResults().omitEmptyStrings().splitToList(groupOwnerships.get(group).getAsString())

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LocalGroupOwnershipService.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.gobblin.service;
+
+import com.google.gson.JsonObject;
+import com.typesafe.config.Config;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import com.google.gson.JsonParser;
+
+
+public class LocalGroupOwnershipService extends GroupOwnershipService {
+  public static final String GROUP_MEMBER_LIST = ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "groupMembers.path";
+
+  JsonObject groupOwnerships;
+
+  public LocalGroupOwnershipService(Config config) {
+    Path groupOwnershipFilePath = new Path(config.getString(GROUP_MEMBER_LIST));
+    FSDataInputStream in = null;
+    try {
+      FileSystem fs = FileSystem.get(new Configuration());
+      in = fs.open(groupOwnershipFilePath);
+      String jsonString = IOUtils.toString(in, Charset.defaultCharset());
+      JsonParser parser = new JsonParser();
+      this.groupOwnerships = parser.parse(jsonString).getAsJsonObject();
+      in.close();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public boolean isMemberOfGroup(List<ServiceRequester> serviceRequesters, String group) {
+    if (this.groupOwnerships.has(group)) {
+      String member = extractRequesterUser(serviceRequesters);
+      List<String> groupMembers = Arrays.asList(groupOwnerships.get(group).getAsString().split(","));
+      return groupMembers.contains(member);
+    }
+    return false;
+  }
+
+  private static String extractRequesterUser(List<ServiceRequester> requesterList) {

Review comment:
       Probably this method can go into the abstract class and presumably, will be needed for other implementations too.

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2Resource.java
##########
@@ -200,4 +210,29 @@ private Properties getHeaders() {
     }
     return headerProperties;
   }
+
+  public void checkRequester(
+      RequesterService requesterService, FlowConfig originalFlowConfig, List<ServiceRequester> requesterList) {
+    if (requesterList == null) {
+      return;
+    }
+
+    try {
+      String serializedOriginalRequesterList = originalFlowConfig.getProperties().get(RequesterService.REQUESTER_LIST);
+      if (serializedOriginalRequesterList != null) {
+        List<ServiceRequester> originalRequesterList = RequesterService.deserialize(serializedOriginalRequesterList);
+        String owningGroup = originalFlowConfig.getProperties().getOrDefault("owning_group", "");
+        // check that the requester is part of the owning group
+        if (!owningGroup.isEmpty()) {

Review comment:
       So is the expected behavior that if owningGroup is specified, only members of that group are allowed to perform the request? What if we have a group with a single member that created a flow and that member is no longer in the organization?

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/LocalGroupOwnershipService.java
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.gobblin.service;
+
+import com.google.gson.JsonObject;
+import com.typesafe.config.Config;
+import java.io.IOException;
+import java.nio.charset.Charset;
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import com.google.gson.JsonParser;
+
+
+public class LocalGroupOwnershipService extends GroupOwnershipService {
+  public static final String GROUP_MEMBER_LIST = ServiceConfigKeys.GOBBLIN_SERVICE_PREFIX + "groupMembers.path";
+
+  JsonObject groupOwnerships;
+
+  public LocalGroupOwnershipService(Config config) {
+    Path groupOwnershipFilePath = new Path(config.getString(GROUP_MEMBER_LIST));
+    FSDataInputStream in = null;
+    try {
+      FileSystem fs = FileSystem.get(new Configuration());
+      in = fs.open(groupOwnershipFilePath);
+      String jsonString = IOUtils.toString(in, Charset.defaultCharset());
+      JsonParser parser = new JsonParser();
+      this.groupOwnerships = parser.parse(jsonString).getAsJsonObject();
+      in.close();

Review comment:
       try with resources to avoid having to call in.close()?

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowConfigsV2Resource.java
##########
@@ -200,4 +210,29 @@ private Properties getHeaders() {
     }
     return headerProperties;
   }
+
+  public void checkRequester(
+      RequesterService requesterService, FlowConfig originalFlowConfig, List<ServiceRequester> requesterList) {
+    if (requesterList == null) {
+      return;
+    }
+
+    try {
+      String serializedOriginalRequesterList = originalFlowConfig.getProperties().get(RequesterService.REQUESTER_LIST);
+      if (serializedOriginalRequesterList != null) {
+        List<ServiceRequester> originalRequesterList = RequesterService.deserialize(serializedOriginalRequesterList);
+        String owningGroup = originalFlowConfig.getProperties().getOrDefault("owning_group", "");
+        // check that the requester is part of the owning group
+        if (!owningGroup.isEmpty()) {

Review comment:
       We should accommodate an "admin" mode where a user with admin privileges can also make a request.




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