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/12 22:28:07 UTC

[GitHub] [incubator-gobblin] jack-moseley opened a new pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

jack-moseley opened a new pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124


   Dear Gobblin maintainers,
   
   Please accept this PR. I understand that it will not be reviewed until I have checked off all the steps below!
   
   
   ### JIRA
   - [x] My PR addresses the following [Gobblin JIRA](https://issues.apache.org/jira/browse/GOBBLIN/) issues and references them in the PR title. For example, "[GOBBLIN-XXX] My Gobblin PR"
       - https://issues.apache.org/jira/browse/GOBBLIN-1061
   
   
   ### Description
   - [x] Here are some details about my PR, including screenshots (if applicable):
   - Modify helix instance name to have the current host name appended to it
   - Add option where, if true, all requests sent to create/update/delete a flowConfig/flowExecution must be sent to the leader node. If they are not, an error will be thrown pointing to the leader URL.
   - Add retry in clients for flowConfig/flowExecution, where if the above error is thrown, the client will retry by sending the same request directly to the leader node.
   
   
   ### Tests
   - [x] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   Tested in gaas locally
   
   
   ### Commits
   - [x] My commits all reference JIRA issues in their subject lines, and I have squashed multiple commits if they address the same issue. In addition, my commits follow the guidelines from "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)":
       1. Subject is separated from body by a blank line
       2. Subject is limited to 50 characters
       3. Subject does not end with a period
       4. Subject uses the imperative mood ("add", not "adding")
       5. Body wraps at 72 characters
       6. Body explains "what" and "why", not "how"
   
   


----------------------------------------------------------------
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] [incubator-gobblin] sv2000 commented on pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#issuecomment-714842437


   @jack-moseley there seems to be some merge conflict here. Can you check?


----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r505902908



##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResource.java
##########
@@ -52,10 +55,19 @@
 public class FlowExecutionResource extends ComplexKeyResourceTemplate<FlowStatusId, EmptyRecord, FlowExecution> {

Review comment:
       I guess that is possible, at least it will be the same pattern between flow config and flow execution then, and we can pass the helixmanager/eventbus directly to it instead of injecting them.




----------------------------------------------------------------
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] [incubator-gobblin] asfgit closed pull request #3124: [GOBBLIN-1285] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124


   


----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#issuecomment-712430257


   Added commit to fix findbugs and tests.
   
   Also checking in reordered restli snapshot.json files, I think there was a restli change where these are in alphabetical order when generated, so they will keep showing up as modified in git after building unless we check in the updated 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] [incubator-gobblin] jack-moseley commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504216140



##########
File path: gobblin-runtime/build.gradle
##########
@@ -66,6 +66,7 @@ dependencies {
   compile externalDependency.gson
   compile externalDependency.guava
   compile externalDependency.guice
+  compile externalDependency.helix

Review comment:
       I did try a lot of different locations for these things... the problem with doing that is that `FlowExecutionResource` needs `FlowStatusGenerator` as well, and I can't add gobblin-service as a dependency because gobblin-service depends on gobblin-restli.
   
   Maybe I can try moving `FlowStatusGenerator` to gobblin-restli, that seems like it should be possible.




----------------------------------------------------------------
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] [incubator-gobblin] shirshanka commented on pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
shirshanka commented on pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#issuecomment-718238932


   @jack-moseley : do you have the right JIRA on the PR? 


----------------------------------------------------------------
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] [incubator-gobblin] sv2000 commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504114249



##########
File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/HelixLeaderUtils.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtime.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.LiveInstance;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.linkedin.data.DataMap;
+import com.linkedin.restli.common.HttpStatus;
+import com.linkedin.restli.server.RestLiServiceException;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Utils for storing/parsing helix URL in the helix instance name
+ */
+@Slf4j
+public class HelixLeaderUtils {
+  public static String HELIX_INSTANCE_NAME_SEPARATOR = "@";
+
+  /**
+   */
+  private static String getUrlFromHelixInstanceName(String helixInstanceName) {
+    if (!helixInstanceName.contains(HELIX_INSTANCE_NAME_SEPARATOR)) {
+      return null;
+    } else {
+      return helixInstanceName.substring(helixInstanceName.indexOf(HELIX_INSTANCE_NAME_SEPARATOR) + 1);
+    }
+  }
+
+  private static String getLeaderUrl(HelixManager helixManager) {
+    PropertyKey key = helixManager.getHelixDataAccessor().keyBuilder().controllerLeader();
+    LiveInstance leader = helixManager.getHelixDataAccessor().getProperty(key);
+    return getUrlFromHelixInstanceName(leader.getInstanceName());
+  }
+
+  /**
+   * If this host is not the leader, throw a {@link RestLiServiceException}, and include the URL of the leader host in
+   * the message and in the errorDetails under the key {@link ServiceConfigKeys#LEADER_URL}.
+   */
+  public static void throwErrorIfNotLeader(Optional<HelixManager> helixManager)  {
+    if (helixManager.isPresent() && !helixManager.get().isLeader()) {
+      String leaderUrl = getLeaderUrl(helixManager.get());
+      if (leaderUrl == null) {
+        return;
+      }
+      RestLiServiceException exception = new RestLiServiceException(HttpStatus.S_400_BAD_REQUEST, "Request must be sent to leader node at URL " + leaderUrl);
+      exception.setErrorDetails(new DataMap(ImmutableMap.of(ServiceConfigKeys.LEADER_URL, leaderUrl)));
+      throw exception;
+    }
+  }
+
+  /**
+   * Build helix instance name by getting {@link org.apache.gobblin.service.ServiceConfigKeys#HELIX_INSTANCE_NAME_KEY}
+   * and appending the host, port, and service name with a separator
+   */
+  public static String buildHelixInstanceName(Config config, String defaultInstanceName) {
+    String helixInstanceName = ConfigUtils.getString(config, ServiceConfigKeys.HELIX_INSTANCE_NAME_KEY, defaultInstanceName);
+
+    String url = "";
+    try {
+      url = HELIX_INSTANCE_NAME_SEPARATOR + "https://" + InetAddress.getLocalHost().getHostName() + ":" + ConfigUtils.getString(config, ServiceConfigKeys.SERVICE_PORT, "")

Review comment:
       Also - should we always assume https? Can this be inferred from service configs i.e. http vs https.




----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r505936225



##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResource.java
##########
@@ -52,10 +55,19 @@
 public class FlowExecutionResource extends ComplexKeyResourceTemplate<FlowStatusId, EmptyRecord, FlowExecution> {

Review comment:
       Added a change to:
   - Add `FlowExecutionResourceHandler`
   - Add `GobblinServiceFlowExecutionResourceHandler` which just calls the underlying one, except for delete where it does the leadership check
   - Merge the `HelixLeaderUtils` methods into `HelixUtils` in gobblin-service and remove the extra helix dependencies added earlier
   - Pass eventbus/helixmanager/flowstatusgenerator directly to flowExecutionResourceHandler instead of injecting them, and also switch to building flowstatusgenerator here instead of internally (since there is no reason for it to be internal)




----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504369163



##########
File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/HelixLeaderUtils.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtime.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.LiveInstance;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.linkedin.data.DataMap;
+import com.linkedin.restli.common.HttpStatus;
+import com.linkedin.restli.server.RestLiServiceException;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Utils for storing/parsing helix URL in the helix instance name
+ */
+@Slf4j
+public class HelixLeaderUtils {

Review comment:
       Moved to gobblin-restli




----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley closed pull request #3124: [GOBBLIN-1285] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley closed pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124


   


----------------------------------------------------------------
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] [incubator-gobblin] sv2000 commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504078343



##########
File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/HelixLeaderUtils.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtime.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.LiveInstance;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.linkedin.data.DataMap;
+import com.linkedin.restli.common.HttpStatus;
+import com.linkedin.restli.server.RestLiServiceException;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Utils for storing/parsing helix URL in the helix instance name
+ */
+@Slf4j
+public class HelixLeaderUtils {
+  public static String HELIX_INSTANCE_NAME_SEPARATOR = "@";
+
+  /**
+   */
+  private static String getUrlFromHelixInstanceName(String helixInstanceName) {
+    if (!helixInstanceName.contains(HELIX_INSTANCE_NAME_SEPARATOR)) {
+      return null;
+    } else {
+      return helixInstanceName.substring(helixInstanceName.indexOf(HELIX_INSTANCE_NAME_SEPARATOR) + 1);
+    }
+  }
+
+  private static String getLeaderUrl(HelixManager helixManager) {
+    PropertyKey key = helixManager.getHelixDataAccessor().keyBuilder().controllerLeader();
+    LiveInstance leader = helixManager.getHelixDataAccessor().getProperty(key);
+    return getUrlFromHelixInstanceName(leader.getInstanceName());
+  }
+
+  /**
+   * If this host is not the leader, throw a {@link RestLiServiceException}, and include the URL of the leader host in
+   * the message and in the errorDetails under the key {@link ServiceConfigKeys#LEADER_URL}.
+   */
+  public static void throwErrorIfNotLeader(Optional<HelixManager> helixManager)  {
+    if (helixManager.isPresent() && !helixManager.get().isLeader()) {
+      String leaderUrl = getLeaderUrl(helixManager.get());
+      if (leaderUrl == null) {
+        return;

Review comment:
       Should we throw an error here?

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/main/java/org/apache/gobblin/service/FlowClientUtils.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.net.URI;
+import java.net.URISyntaxException;
+
+import com.linkedin.r2.RemoteInvocationException;
+import com.linkedin.r2.message.RequestContext;
+import com.linkedin.restli.client.Request;
+import com.linkedin.restli.client.Response;
+import com.linkedin.restli.client.RestClient;
+import com.linkedin.restli.client.RestLiResponseException;
+
+
+/**
+ * Utils to be used by clients
+ */
+public class FlowClientUtils {
+  /**
+   * Send a restli {@link Request} to the server through a {@link RestClient}, but if the request is rejected due to not
+   * being sent to a leader node, get the leader node from the errorDetails and retry the request with that node by setting
+   * the D2-Hint-TargetService attribute.
+   * @param restClient rest client to use to send the request
+   * @param request request to send
+   * @param primaryResource resource part of the request URL (e.g. flowconfigsV2, which can be taken from
+   *        {@link FlowconfigsV2RequestBuilders#getPrimaryResource()}
+   * @return {@link Response} returned from the request
+   * @throws RemoteInvocationException
+   */
+  public static Response<?> sendRequestWithRetry(RestClient restClient, Request<?> request, String primaryResource) throws RemoteInvocationException {

Review comment:
       Can we add unit tests? We should be able to add unit tests in GobblinServiceHATest. 




----------------------------------------------------------------
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] [incubator-gobblin] sv2000 commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r505677256



##########
File path: gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceRedirectTest.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.modules.core;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Maps;
+import com.linkedin.data.template.StringMap;
+import com.linkedin.r2.transport.http.client.HttpClientFactory;
+import com.linkedin.restli.client.RestLiResponseException;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlJobStatusStateStoreFactory;
+import org.apache.gobblin.metastore.testing.ITestMetastoreDatabase;
+import org.apache.gobblin.metastore.testing.TestMetastoreDatabaseFactory;
+import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
+import org.apache.gobblin.service.FlowConfig;
+import org.apache.gobblin.service.FlowConfigClient;
+import org.apache.gobblin.service.FlowId;
+import org.apache.gobblin.service.Schedule;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.utils.HelixUtils;
+import org.apache.gobblin.service.monitoring.FsJobStatusRetriever;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Test
+public class GobblinServiceRedirectTest {
+
+  private static final Logger logger = LoggerFactory.getLogger(GobblinServiceRedirectTest.class);
+
+  private static final String QUARTZ_INSTANCE_NAME = "org.quartz.scheduler.instanceName";
+  private static final String QUARTZ_THREAD_POOL_COUNT = "org.quartz.threadPool.threadCount";
+
+  private static final String COMMON_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreCommon/";
+
+  private static final String NODE_1_SERVICE_WORK_DIR = "/tmp/serviceWorkDirNode1/";
+  private static final String NODE_1_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreNode1/";
+  private static final String NODE_1_TOPOLOGY_SPEC_STORE_DIR = "/tmp/serviceCoreNode1/topologyTestSpecStoreNode1";
+  private static final String NODE_1_FLOW_SPEC_STORE_DIR = "/tmp/serviceCoreCommon/flowTestSpecStore";
+  private static final String NODE_1_JOB_STATUS_STATE_STORE_DIR = "/tmp/serviceCoreNode1/fsJobStatusRetriever";
+
+  private static final String NODE_2_SERVICE_WORK_DIR = "/tmp/serviceWorkDirNode2/";
+  private static final String NODE_2_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreNode2/";
+  private static final String NODE_2_TOPOLOGY_SPEC_STORE_DIR = "/tmp/serviceCoreNode2/topologyTestSpecStoreNode2";
+  private static final String NODE_2_FLOW_SPEC_STORE_DIR = "/tmp/serviceCoreCommon/flowTestSpecStore";
+  private static final String NODE_2_JOB_STATUS_STATE_STORE_DIR = "/tmp/serviceCoreNode2/fsJobStatusRetriever";
+
+  private static final String TEST_HELIX_CLUSTER_NAME = "testGobblinServiceCluster";
+
+  private static final String TEST_GROUP_NAME_1 = "testGroup1";
+  private static final String TEST_FLOW_NAME_1 = "testFlow1";
+  private static final String TEST_SCHEDULE_1 = "0 1/0 * ? * *";
+  private static final String TEST_TEMPLATE_URI_1 = "FS:///templates/test.template";
+
+  private static final String TEST_GROUP_NAME_2 = "testGroup2";
+  private static final String TEST_FLOW_NAME_2 = "testFlow2";
+  private static final String TEST_SCHEDULE_2 = "0 1/0 * ? * *";
+  private static final String TEST_TEMPLATE_URI_2 = "FS:///templates/test.template";
+
+  private static final String TEST_GOBBLIN_EXECUTOR_NAME = "testGobblinExecutor";
+  private static final String TEST_SOURCE_NAME = "testSource";
+  private static final String TEST_SINK_NAME = "testSink";
+
+  private static final String PORT1 = "1000";
+  private static final String PORT2 = "2000";
+  private static final String PREFIX = "https://";
+  private static final String SERVICE_NAME = "gobblinServiceTest";
+
+  private GobblinServiceManager node1GobblinServiceManager;
+  private FlowConfigClient node1FlowConfigClient;
+
+  private GobblinServiceManager node2GobblinServiceManager;
+  private FlowConfigClient node2FlowConfigClient;
+
+  private TestingServer testingZKServer;
+
+  private Properties node1ServiceCoreProperties;
+  private Properties node2ServiceCoreProperties;
+
+  @BeforeClass
+  public void setup() throws Exception {
+    // Clean up common Flow Spec Dir
+    cleanUpDir(COMMON_SPEC_STORE_PARENT_DIR);
+
+    // Clean up work dir for Node 1
+    cleanUpDir(NODE_1_SERVICE_WORK_DIR);
+    cleanUpDir(NODE_1_SPEC_STORE_PARENT_DIR);
+
+    // Clean up work dir for Node 2
+    cleanUpDir(NODE_2_SERVICE_WORK_DIR);
+    cleanUpDir(NODE_2_SPEC_STORE_PARENT_DIR);
+
+    // Use a random ZK port
+    this.testingZKServer = new TestingServer(-1);
+    logger.info("Testing ZK Server listening on: " + testingZKServer.getConnectString());
+    HelixUtils.createGobblinHelixCluster(testingZKServer.getConnectString(), TEST_HELIX_CLUSTER_NAME);
+
+    ITestMetastoreDatabase testMetastoreDatabase = TestMetastoreDatabaseFactory.get();
+
+    Properties commonServiceCoreProperties = new Properties();
+    commonServiceCoreProperties.put(ServiceConfigKeys.ZK_CONNECTION_STRING_KEY, testingZKServer.getConnectString());
+    commonServiceCoreProperties.put(ServiceConfigKeys.HELIX_CLUSTER_NAME_KEY, TEST_HELIX_CLUSTER_NAME);
+    commonServiceCoreProperties.put(ServiceConfigKeys.HELIX_INSTANCE_NAME_KEY, "GaaS_" + UUID.randomUUID().toString());
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_TOPOLOGY_NAMES_KEY , TEST_GOBBLIN_EXECUTOR_NAME);
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".description",
+        "StandaloneTestExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".version",
+        "1");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".uri",
+        "gobblinExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".specExecutorInstance",
+        "org.gobblin.service.InMemorySpecExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".specExecInstance.capabilities",
+        TEST_SOURCE_NAME + ":" + TEST_SINK_NAME);
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_USER_KEY, "testUser");
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_PASSWORD_KEY, "testPassword");
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_URL_KEY, testMetastoreDatabase.getJdbcUrl());
+    commonServiceCoreProperties.put("zookeeper.connect", testingZKServer.getConnectString());
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_FACTORY_CLASS_KEY, MysqlJobStatusStateStoreFactory.class.getName());
+    commonServiceCoreProperties.put(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_STATUS_MONITOR_ENABLED_KEY, false);
+
+    commonServiceCoreProperties.put(ServiceConfigKeys.FORCE_LEADER, true);
+    commonServiceCoreProperties.put(ServiceConfigKeys.SERVICE_URL_PREFIX, PREFIX);
+    commonServiceCoreProperties.put(ServiceConfigKeys.SERVICE_NAME, SERVICE_NAME);
+
+    node1ServiceCoreProperties = new Properties();
+    node1ServiceCoreProperties.putAll(commonServiceCoreProperties);
+    node1ServiceCoreProperties.put(ConfigurationKeys.TOPOLOGYSPEC_STORE_DIR_KEY, NODE_1_TOPOLOGY_SPEC_STORE_DIR);
+    node1ServiceCoreProperties.put(FlowCatalog.FLOWSPEC_STORE_DIR_KEY, NODE_1_FLOW_SPEC_STORE_DIR);
+    node1ServiceCoreProperties.put(FsJobStatusRetriever.CONF_PREFIX + "." + ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY, NODE_1_JOB_STATUS_STATE_STORE_DIR);
+    node1ServiceCoreProperties.put(QUARTZ_INSTANCE_NAME, "QuartzScheduler1");
+    node1ServiceCoreProperties.put(QUARTZ_THREAD_POOL_COUNT, 3);
+    node1ServiceCoreProperties.put(ServiceConfigKeys.SERVICE_PORT, PORT1);
+
+    node2ServiceCoreProperties = new Properties();
+    node2ServiceCoreProperties.putAll(commonServiceCoreProperties);
+    node2ServiceCoreProperties.put(ConfigurationKeys.TOPOLOGYSPEC_STORE_DIR_KEY, NODE_2_TOPOLOGY_SPEC_STORE_DIR);
+    node2ServiceCoreProperties.put(FlowCatalog.FLOWSPEC_STORE_DIR_KEY, NODE_2_FLOW_SPEC_STORE_DIR);
+    node2ServiceCoreProperties.put(FsJobStatusRetriever.CONF_PREFIX + "." + ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY, NODE_2_JOB_STATUS_STATE_STORE_DIR);
+    node2ServiceCoreProperties.put(QUARTZ_INSTANCE_NAME, "QuartzScheduler2");
+    node2ServiceCoreProperties.put(QUARTZ_THREAD_POOL_COUNT, 3);
+    node2ServiceCoreProperties.put(ServiceConfigKeys.SERVICE_PORT, PORT2);

Review comment:
       Same comment as above.

##########
File path: gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceRedirectTest.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.modules.core;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Maps;
+import com.linkedin.data.template.StringMap;
+import com.linkedin.r2.transport.http.client.HttpClientFactory;
+import com.linkedin.restli.client.RestLiResponseException;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlJobStatusStateStoreFactory;
+import org.apache.gobblin.metastore.testing.ITestMetastoreDatabase;
+import org.apache.gobblin.metastore.testing.TestMetastoreDatabaseFactory;
+import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
+import org.apache.gobblin.service.FlowConfig;
+import org.apache.gobblin.service.FlowConfigClient;
+import org.apache.gobblin.service.FlowId;
+import org.apache.gobblin.service.Schedule;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.utils.HelixUtils;
+import org.apache.gobblin.service.monitoring.FsJobStatusRetriever;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Test
+public class GobblinServiceRedirectTest {
+
+  private static final Logger logger = LoggerFactory.getLogger(GobblinServiceRedirectTest.class);
+
+  private static final String QUARTZ_INSTANCE_NAME = "org.quartz.scheduler.instanceName";
+  private static final String QUARTZ_THREAD_POOL_COUNT = "org.quartz.threadPool.threadCount";
+
+  private static final String COMMON_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreCommon/";
+
+  private static final String NODE_1_SERVICE_WORK_DIR = "/tmp/serviceWorkDirNode1/";
+  private static final String NODE_1_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreNode1/";
+  private static final String NODE_1_TOPOLOGY_SPEC_STORE_DIR = "/tmp/serviceCoreNode1/topologyTestSpecStoreNode1";
+  private static final String NODE_1_FLOW_SPEC_STORE_DIR = "/tmp/serviceCoreCommon/flowTestSpecStore";
+  private static final String NODE_1_JOB_STATUS_STATE_STORE_DIR = "/tmp/serviceCoreNode1/fsJobStatusRetriever";
+
+  private static final String NODE_2_SERVICE_WORK_DIR = "/tmp/serviceWorkDirNode2/";
+  private static final String NODE_2_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreNode2/";
+  private static final String NODE_2_TOPOLOGY_SPEC_STORE_DIR = "/tmp/serviceCoreNode2/topologyTestSpecStoreNode2";
+  private static final String NODE_2_FLOW_SPEC_STORE_DIR = "/tmp/serviceCoreCommon/flowTestSpecStore";
+  private static final String NODE_2_JOB_STATUS_STATE_STORE_DIR = "/tmp/serviceCoreNode2/fsJobStatusRetriever";
+
+  private static final String TEST_HELIX_CLUSTER_NAME = "testGobblinServiceCluster";
+
+  private static final String TEST_GROUP_NAME_1 = "testGroup1";
+  private static final String TEST_FLOW_NAME_1 = "testFlow1";
+  private static final String TEST_SCHEDULE_1 = "0 1/0 * ? * *";
+  private static final String TEST_TEMPLATE_URI_1 = "FS:///templates/test.template";
+
+  private static final String TEST_GROUP_NAME_2 = "testGroup2";
+  private static final String TEST_FLOW_NAME_2 = "testFlow2";
+  private static final String TEST_SCHEDULE_2 = "0 1/0 * ? * *";
+  private static final String TEST_TEMPLATE_URI_2 = "FS:///templates/test.template";
+
+  private static final String TEST_GOBBLIN_EXECUTOR_NAME = "testGobblinExecutor";
+  private static final String TEST_SOURCE_NAME = "testSource";
+  private static final String TEST_SINK_NAME = "testSink";
+
+  private static final String PORT1 = "1000";
+  private static final String PORT2 = "2000";
+  private static final String PREFIX = "https://";
+  private static final String SERVICE_NAME = "gobblinServiceTest";
+
+  private GobblinServiceManager node1GobblinServiceManager;
+  private FlowConfigClient node1FlowConfigClient;
+
+  private GobblinServiceManager node2GobblinServiceManager;
+  private FlowConfigClient node2FlowConfigClient;
+
+  private TestingServer testingZKServer;
+
+  private Properties node1ServiceCoreProperties;
+  private Properties node2ServiceCoreProperties;
+
+  @BeforeClass
+  public void setup() throws Exception {
+    // Clean up common Flow Spec Dir
+    cleanUpDir(COMMON_SPEC_STORE_PARENT_DIR);
+
+    // Clean up work dir for Node 1
+    cleanUpDir(NODE_1_SERVICE_WORK_DIR);
+    cleanUpDir(NODE_1_SPEC_STORE_PARENT_DIR);
+
+    // Clean up work dir for Node 2
+    cleanUpDir(NODE_2_SERVICE_WORK_DIR);
+    cleanUpDir(NODE_2_SPEC_STORE_PARENT_DIR);
+
+    // Use a random ZK port
+    this.testingZKServer = new TestingServer(-1);
+    logger.info("Testing ZK Server listening on: " + testingZKServer.getConnectString());
+    HelixUtils.createGobblinHelixCluster(testingZKServer.getConnectString(), TEST_HELIX_CLUSTER_NAME);
+
+    ITestMetastoreDatabase testMetastoreDatabase = TestMetastoreDatabaseFactory.get();
+
+    Properties commonServiceCoreProperties = new Properties();
+    commonServiceCoreProperties.put(ServiceConfigKeys.ZK_CONNECTION_STRING_KEY, testingZKServer.getConnectString());
+    commonServiceCoreProperties.put(ServiceConfigKeys.HELIX_CLUSTER_NAME_KEY, TEST_HELIX_CLUSTER_NAME);
+    commonServiceCoreProperties.put(ServiceConfigKeys.HELIX_INSTANCE_NAME_KEY, "GaaS_" + UUID.randomUUID().toString());
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_TOPOLOGY_NAMES_KEY , TEST_GOBBLIN_EXECUTOR_NAME);
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".description",
+        "StandaloneTestExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".version",
+        "1");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".uri",
+        "gobblinExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".specExecutorInstance",
+        "org.gobblin.service.InMemorySpecExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".specExecInstance.capabilities",
+        TEST_SOURCE_NAME + ":" + TEST_SINK_NAME);
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_USER_KEY, "testUser");
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_PASSWORD_KEY, "testPassword");
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_URL_KEY, testMetastoreDatabase.getJdbcUrl());
+    commonServiceCoreProperties.put("zookeeper.connect", testingZKServer.getConnectString());
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_FACTORY_CLASS_KEY, MysqlJobStatusStateStoreFactory.class.getName());
+    commonServiceCoreProperties.put(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_STATUS_MONITOR_ENABLED_KEY, false);
+
+    commonServiceCoreProperties.put(ServiceConfigKeys.FORCE_LEADER, true);
+    commonServiceCoreProperties.put(ServiceConfigKeys.SERVICE_URL_PREFIX, PREFIX);
+    commonServiceCoreProperties.put(ServiceConfigKeys.SERVICE_NAME, SERVICE_NAME);
+
+    node1ServiceCoreProperties = new Properties();
+    node1ServiceCoreProperties.putAll(commonServiceCoreProperties);
+    node1ServiceCoreProperties.put(ConfigurationKeys.TOPOLOGYSPEC_STORE_DIR_KEY, NODE_1_TOPOLOGY_SPEC_STORE_DIR);
+    node1ServiceCoreProperties.put(FlowCatalog.FLOWSPEC_STORE_DIR_KEY, NODE_1_FLOW_SPEC_STORE_DIR);
+    node1ServiceCoreProperties.put(FsJobStatusRetriever.CONF_PREFIX + "." + ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY, NODE_1_JOB_STATUS_STATE_STORE_DIR);
+    node1ServiceCoreProperties.put(QUARTZ_INSTANCE_NAME, "QuartzScheduler1");
+    node1ServiceCoreProperties.put(QUARTZ_THREAD_POOL_COUNT, 3);
+    node1ServiceCoreProperties.put(ServiceConfigKeys.SERVICE_PORT, PORT1);

Review comment:
       Should this be restliServer.getPort() instead of PORT1?

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResource.java
##########
@@ -52,10 +55,19 @@
 public class FlowExecutionResource extends ComplexKeyResourceTemplate<FlowStatusId, EmptyRecord, FlowExecution> {

Review comment:
       One naive question: Can we create a FlowExecutionResourceHandler interface with impls inside gobblin-service module and inject the handler here? Essentially, replicating the pattern we used for FlowConfigs resource, and making the code uniform. 

##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/HelixLeaderUtils.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.io.UnsupportedEncodingException;
+import java.net.InetAddress;
+import java.net.URLDecoder;
+import java.net.URLEncoder;
+import java.net.UnknownHostException;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.LiveInstance;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.linkedin.data.DataMap;
+import com.linkedin.restli.common.HttpStatus;
+import com.linkedin.restli.server.RestLiServiceException;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Utils for storing/parsing helix URL in the helix instance name
+ */
+@Slf4j
+public class HelixLeaderUtils {
+  public static String HELIX_INSTANCE_NAME_SEPARATOR = "@";
+
+  /**

Review comment:
       Empty Javadoc here. You may want to remove it if not needed.

##########
File path: gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceRedirectTest.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.modules.core;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Maps;
+import com.linkedin.data.template.StringMap;
+import com.linkedin.r2.transport.http.client.HttpClientFactory;
+import com.linkedin.restli.client.RestLiResponseException;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlJobStatusStateStoreFactory;
+import org.apache.gobblin.metastore.testing.ITestMetastoreDatabase;
+import org.apache.gobblin.metastore.testing.TestMetastoreDatabaseFactory;
+import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
+import org.apache.gobblin.service.FlowConfig;
+import org.apache.gobblin.service.FlowConfigClient;
+import org.apache.gobblin.service.FlowId;
+import org.apache.gobblin.service.Schedule;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.utils.HelixUtils;
+import org.apache.gobblin.service.monitoring.FsJobStatusRetriever;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Test
+public class GobblinServiceRedirectTest {
+
+  private static final Logger logger = LoggerFactory.getLogger(GobblinServiceRedirectTest.class);
+
+  private static final String QUARTZ_INSTANCE_NAME = "org.quartz.scheduler.instanceName";
+  private static final String QUARTZ_THREAD_POOL_COUNT = "org.quartz.threadPool.threadCount";
+
+  private static final String COMMON_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreCommon/";
+
+  private static final String NODE_1_SERVICE_WORK_DIR = "/tmp/serviceWorkDirNode1/";

Review comment:
       Can we dynamically generate tmp dirs using Guava Files.createTempDir() method? And then mark the file to be deleted using deleteOnExit(). Hardcoding the location to /tmp has the potential to create flaky tests due to left over state from previous builds in case of un-finished clean up. Also, it may end up interfering with identical locations used in other 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] [incubator-gobblin] sv2000 commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504819940



##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-server/src/main/java/org/apache/gobblin/service/FlowExecutionResource.java
##########
@@ -52,10 +55,19 @@
 public class FlowExecutionResource extends ComplexKeyResourceTemplate<FlowStatusId, EmptyRecord, FlowExecution> {
   private static final Logger LOG = LoggerFactory.getLogger(FlowExecutionResource.class);
   public static final String FLOW_STATUS_GENERATOR_INJECT_NAME = "FlowStatusGenerator";
+  public static final String EVENTBUS_INJECT_NAME = "EventBus";
+  public static final String HELIX_MANAGER_INJECT_NAME = "HelixManager";
+  public static final String FORCE_LEADER_INJECT_NAME = "ForceLeader";
   public static final String MESSAGE_SEPARATOR = ", ";
 
   @Inject @javax.inject.Inject @javax.inject.Named(FLOW_STATUS_GENERATOR_INJECT_NAME)
   FlowStatusGenerator _flowStatusGenerator;
+  @Inject @javax.inject.Inject @javax.inject.Named(EVENTBUS_INJECT_NAME)
+  EventBus eventBus;
+  @Inject @javax.inject.Inject @javax.inject.Named(HELIX_MANAGER_INJECT_NAME)

Review comment:
       It seems like we don't really need a HelixManager instance here and the dependency on helix module can be removed. Can HelixLeaderUtils create a singleton HelixManager instance that it uses to check the leader status? 




----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#issuecomment-718136198


   Switched to PortUtils and squashed commits


----------------------------------------------------------------
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] [incubator-gobblin] sv2000 commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r512334159



##########
File path: gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceRedirectTest.java
##########
@@ -90,8 +90,8 @@
   private static final String TEST_SOURCE_NAME = "testSource";
   private static final String TEST_SINK_NAME = "testSink";
 
-  private static final String PORT1 = "1000";
-  private static final String PORT2 = "2000";
+  private static final String PORT1 = "10000";

Review comment:
       You can checkout PortUtils class that provides a method for handing out random unused ports. Static port values run the risk of port conflicts.




----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504301755



##########
File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/HelixLeaderUtils.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtime.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.LiveInstance;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.linkedin.data.DataMap;
+import com.linkedin.restli.common.HttpStatus;
+import com.linkedin.restli.server.RestLiServiceException;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Utils for storing/parsing helix URL in the helix instance name
+ */
+@Slf4j
+public class HelixLeaderUtils {
+  public static String HELIX_INSTANCE_NAME_SEPARATOR = "@";
+
+  /**
+   */
+  private static String getUrlFromHelixInstanceName(String helixInstanceName) {
+    if (!helixInstanceName.contains(HELIX_INSTANCE_NAME_SEPARATOR)) {
+      return null;
+    } else {
+      return helixInstanceName.substring(helixInstanceName.indexOf(HELIX_INSTANCE_NAME_SEPARATOR) + 1);
+    }
+  }
+
+  private static String getLeaderUrl(HelixManager helixManager) {
+    PropertyKey key = helixManager.getHelixDataAccessor().keyBuilder().controllerLeader();
+    LiveInstance leader = helixManager.getHelixDataAccessor().getProperty(key);
+    return getUrlFromHelixInstanceName(leader.getInstanceName());
+  }
+
+  /**
+   * If this host is not the leader, throw a {@link RestLiServiceException}, and include the URL of the leader host in
+   * the message and in the errorDetails under the key {@link ServiceConfigKeys#LEADER_URL}.
+   */
+  public static void throwErrorIfNotLeader(Optional<HelixManager> helixManager)  {
+    if (helixManager.isPresent() && !helixManager.get().isLeader()) {
+      String leaderUrl = getLeaderUrl(helixManager.get());
+      if (leaderUrl == null) {
+        return;
+      }
+      RestLiServiceException exception = new RestLiServiceException(HttpStatus.S_400_BAD_REQUEST, "Request must be sent to leader node at URL " + leaderUrl);
+      exception.setErrorDetails(new DataMap(ImmutableMap.of(ServiceConfigKeys.LEADER_URL, leaderUrl)));
+      throw exception;
+    }
+  }
+
+  /**
+   * Build helix instance name by getting {@link org.apache.gobblin.service.ServiceConfigKeys#HELIX_INSTANCE_NAME_KEY}
+   * and appending the host, port, and service name with a separator
+   */
+  public static String buildHelixInstanceName(Config config, String defaultInstanceName) {
+    String helixInstanceName = ConfigUtils.getString(config, ServiceConfigKeys.HELIX_INSTANCE_NAME_KEY, defaultInstanceName);
+
+    String url = "";
+    try {
+      url = HELIX_INSTANCE_NAME_SEPARATOR + "https://" + InetAddress.getLocalHost().getHostName() + ":" + ConfigUtils.getString(config, ServiceConfigKeys.SERVICE_PORT, "")

Review comment:
       Do you think it's better to keep it backwards compatible? These configs were not required in the past, I didn't want block gaas from starting up if the version is bumped without adding these configs.




----------------------------------------------------------------
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] [incubator-gobblin] sv2000 commented on pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#issuecomment-709447320


   @jack-moseley Can you check on the travis build failures 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] [incubator-gobblin] jack-moseley commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r505931570



##########
File path: gobblin-service/src/test/java/org/apache/gobblin/service/modules/core/GobblinServiceRedirectTest.java
##########
@@ -0,0 +1,301 @@
+/*
+ * 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.modules.core;
+
+import java.io.File;
+import java.net.InetAddress;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.curator.test.TestingServer;
+import org.apache.hadoop.fs.Path;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testng.Assert;
+import org.testng.annotations.AfterClass;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Maps;
+import com.linkedin.data.template.StringMap;
+import com.linkedin.r2.transport.http.client.HttpClientFactory;
+import com.linkedin.restli.client.RestLiResponseException;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metastore.MysqlJobStatusStateStoreFactory;
+import org.apache.gobblin.metastore.testing.ITestMetastoreDatabase;
+import org.apache.gobblin.metastore.testing.TestMetastoreDatabaseFactory;
+import org.apache.gobblin.runtime.spec_catalog.FlowCatalog;
+import org.apache.gobblin.service.FlowConfig;
+import org.apache.gobblin.service.FlowConfigClient;
+import org.apache.gobblin.service.FlowId;
+import org.apache.gobblin.service.Schedule;
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.service.modules.utils.HelixUtils;
+import org.apache.gobblin.service.monitoring.FsJobStatusRetriever;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+@Test
+public class GobblinServiceRedirectTest {
+
+  private static final Logger logger = LoggerFactory.getLogger(GobblinServiceRedirectTest.class);
+
+  private static final String QUARTZ_INSTANCE_NAME = "org.quartz.scheduler.instanceName";
+  private static final String QUARTZ_THREAD_POOL_COUNT = "org.quartz.threadPool.threadCount";
+
+  private static final String COMMON_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreCommon/";
+
+  private static final String NODE_1_SERVICE_WORK_DIR = "/tmp/serviceWorkDirNode1/";
+  private static final String NODE_1_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreNode1/";
+  private static final String NODE_1_TOPOLOGY_SPEC_STORE_DIR = "/tmp/serviceCoreNode1/topologyTestSpecStoreNode1";
+  private static final String NODE_1_FLOW_SPEC_STORE_DIR = "/tmp/serviceCoreCommon/flowTestSpecStore";
+  private static final String NODE_1_JOB_STATUS_STATE_STORE_DIR = "/tmp/serviceCoreNode1/fsJobStatusRetriever";
+
+  private static final String NODE_2_SERVICE_WORK_DIR = "/tmp/serviceWorkDirNode2/";
+  private static final String NODE_2_SPEC_STORE_PARENT_DIR = "/tmp/serviceCoreNode2/";
+  private static final String NODE_2_TOPOLOGY_SPEC_STORE_DIR = "/tmp/serviceCoreNode2/topologyTestSpecStoreNode2";
+  private static final String NODE_2_FLOW_SPEC_STORE_DIR = "/tmp/serviceCoreCommon/flowTestSpecStore";
+  private static final String NODE_2_JOB_STATUS_STATE_STORE_DIR = "/tmp/serviceCoreNode2/fsJobStatusRetriever";
+
+  private static final String TEST_HELIX_CLUSTER_NAME = "testGobblinServiceCluster";
+
+  private static final String TEST_GROUP_NAME_1 = "testGroup1";
+  private static final String TEST_FLOW_NAME_1 = "testFlow1";
+  private static final String TEST_SCHEDULE_1 = "0 1/0 * ? * *";
+  private static final String TEST_TEMPLATE_URI_1 = "FS:///templates/test.template";
+
+  private static final String TEST_GROUP_NAME_2 = "testGroup2";
+  private static final String TEST_FLOW_NAME_2 = "testFlow2";
+  private static final String TEST_SCHEDULE_2 = "0 1/0 * ? * *";
+  private static final String TEST_TEMPLATE_URI_2 = "FS:///templates/test.template";
+
+  private static final String TEST_GOBBLIN_EXECUTOR_NAME = "testGobblinExecutor";
+  private static final String TEST_SOURCE_NAME = "testSource";
+  private static final String TEST_SINK_NAME = "testSink";
+
+  private static final String PORT1 = "1000";
+  private static final String PORT2 = "2000";
+  private static final String PREFIX = "https://";
+  private static final String SERVICE_NAME = "gobblinServiceTest";
+
+  private GobblinServiceManager node1GobblinServiceManager;
+  private FlowConfigClient node1FlowConfigClient;
+
+  private GobblinServiceManager node2GobblinServiceManager;
+  private FlowConfigClient node2FlowConfigClient;
+
+  private TestingServer testingZKServer;
+
+  private Properties node1ServiceCoreProperties;
+  private Properties node2ServiceCoreProperties;
+
+  @BeforeClass
+  public void setup() throws Exception {
+    // Clean up common Flow Spec Dir
+    cleanUpDir(COMMON_SPEC_STORE_PARENT_DIR);
+
+    // Clean up work dir for Node 1
+    cleanUpDir(NODE_1_SERVICE_WORK_DIR);
+    cleanUpDir(NODE_1_SPEC_STORE_PARENT_DIR);
+
+    // Clean up work dir for Node 2
+    cleanUpDir(NODE_2_SERVICE_WORK_DIR);
+    cleanUpDir(NODE_2_SPEC_STORE_PARENT_DIR);
+
+    // Use a random ZK port
+    this.testingZKServer = new TestingServer(-1);
+    logger.info("Testing ZK Server listening on: " + testingZKServer.getConnectString());
+    HelixUtils.createGobblinHelixCluster(testingZKServer.getConnectString(), TEST_HELIX_CLUSTER_NAME);
+
+    ITestMetastoreDatabase testMetastoreDatabase = TestMetastoreDatabaseFactory.get();
+
+    Properties commonServiceCoreProperties = new Properties();
+    commonServiceCoreProperties.put(ServiceConfigKeys.ZK_CONNECTION_STRING_KEY, testingZKServer.getConnectString());
+    commonServiceCoreProperties.put(ServiceConfigKeys.HELIX_CLUSTER_NAME_KEY, TEST_HELIX_CLUSTER_NAME);
+    commonServiceCoreProperties.put(ServiceConfigKeys.HELIX_INSTANCE_NAME_KEY, "GaaS_" + UUID.randomUUID().toString());
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_TOPOLOGY_NAMES_KEY , TEST_GOBBLIN_EXECUTOR_NAME);
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".description",
+        "StandaloneTestExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".version",
+        "1");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".uri",
+        "gobblinExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".specExecutorInstance",
+        "org.gobblin.service.InMemorySpecExecutor");
+    commonServiceCoreProperties.put(ServiceConfigKeys.TOPOLOGY_FACTORY_PREFIX +  TEST_GOBBLIN_EXECUTOR_NAME + ".specExecInstance.capabilities",
+        TEST_SOURCE_NAME + ":" + TEST_SINK_NAME);
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_USER_KEY, "testUser");
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_PASSWORD_KEY, "testPassword");
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_DB_URL_KEY, testMetastoreDatabase.getJdbcUrl());
+    commonServiceCoreProperties.put("zookeeper.connect", testingZKServer.getConnectString());
+    commonServiceCoreProperties.put(ConfigurationKeys.STATE_STORE_FACTORY_CLASS_KEY, MysqlJobStatusStateStoreFactory.class.getName());
+    commonServiceCoreProperties.put(ServiceConfigKeys.GOBBLIN_SERVICE_JOB_STATUS_MONITOR_ENABLED_KEY, false);
+
+    commonServiceCoreProperties.put(ServiceConfigKeys.FORCE_LEADER, true);
+    commonServiceCoreProperties.put(ServiceConfigKeys.SERVICE_URL_PREFIX, PREFIX);
+    commonServiceCoreProperties.put(ServiceConfigKeys.SERVICE_NAME, SERVICE_NAME);
+
+    node1ServiceCoreProperties = new Properties();
+    node1ServiceCoreProperties.putAll(commonServiceCoreProperties);
+    node1ServiceCoreProperties.put(ConfigurationKeys.TOPOLOGYSPEC_STORE_DIR_KEY, NODE_1_TOPOLOGY_SPEC_STORE_DIR);
+    node1ServiceCoreProperties.put(FlowCatalog.FLOWSPEC_STORE_DIR_KEY, NODE_1_FLOW_SPEC_STORE_DIR);
+    node1ServiceCoreProperties.put(FsJobStatusRetriever.CONF_PREFIX + "." + ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY, NODE_1_JOB_STATUS_STATE_STORE_DIR);
+    node1ServiceCoreProperties.put(QUARTZ_INSTANCE_NAME, "QuartzScheduler1");
+    node1ServiceCoreProperties.put(QUARTZ_THREAD_POOL_COUNT, 3);
+    node1ServiceCoreProperties.put(ServiceConfigKeys.SERVICE_PORT, PORT1);

Review comment:
       Can't really do it that way since the restliServer/port is not generated until gobblinservicemanager is instantiated (which already uses this property). Doing it this way just predefines port that will be used by each manager instead of it being random.




----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504369279



##########
File path: gobblin-runtime/build.gradle
##########
@@ -66,6 +66,7 @@ dependencies {
   compile externalDependency.gson
   compile externalDependency.guava
   compile externalDependency.guice
+  compile externalDependency.helix

Review comment:
       Moved the killflow logic to flowExecutionResource so it's not necessary.




----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley closed pull request #3124: [GOBBLIN-1285] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley closed pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124


   


----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on pull request #3124: [GOBBLIN-1285] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#issuecomment-718240470


   @shirshanka good catch, I got the wrong jira somehow. Updated the title and description


----------------------------------------------------------------
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] [incubator-gobblin] sv2000 commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504333555



##########
File path: gobblin-runtime/src/main/java/org/apache/gobblin/service/monitoring/FlowStatusGenerator.java
##########
@@ -176,6 +182,9 @@ private boolean isJobRunning(JobStatus jobStatus) {
    * Send kill request for the given flow
    */
   public void killFlow(String flowGroup, String flowName, Long flowExecutionId) {

Review comment:
       Quick question: Why is there a killFlow() method in FlowStatusGenerator? Can this method be relocated into FlowExecutionResource which seems like the right location? Looks like you only want the eventBus instance in FlowExecutionResource, which can be injected the same way that FlowStatusGenerator is.




----------------------------------------------------------------
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] [incubator-gobblin] jack-moseley commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
jack-moseley commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r504369391



##########
File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/HelixLeaderUtils.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtime.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.LiveInstance;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.linkedin.data.DataMap;
+import com.linkedin.restli.common.HttpStatus;
+import com.linkedin.restli.server.RestLiServiceException;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Utils for storing/parsing helix URL in the helix instance name
+ */
+@Slf4j
+public class HelixLeaderUtils {
+  public static String HELIX_INSTANCE_NAME_SEPARATOR = "@";
+
+  /**
+   */
+  private static String getUrlFromHelixInstanceName(String helixInstanceName) {
+    if (!helixInstanceName.contains(HELIX_INSTANCE_NAME_SEPARATOR)) {
+      return null;
+    } else {
+      return helixInstanceName.substring(helixInstanceName.indexOf(HELIX_INSTANCE_NAME_SEPARATOR) + 1);
+    }
+  }
+
+  private static String getLeaderUrl(HelixManager helixManager) {
+    PropertyKey key = helixManager.getHelixDataAccessor().keyBuilder().controllerLeader();
+    LiveInstance leader = helixManager.getHelixDataAccessor().getProperty(key);
+    return getUrlFromHelixInstanceName(leader.getInstanceName());
+  }
+
+  /**
+   * If this host is not the leader, throw a {@link RestLiServiceException}, and include the URL of the leader host in
+   * the message and in the errorDetails under the key {@link ServiceConfigKeys#LEADER_URL}.
+   */
+  public static void throwErrorIfNotLeader(Optional<HelixManager> helixManager)  {
+    if (helixManager.isPresent() && !helixManager.get().isLeader()) {
+      String leaderUrl = getLeaderUrl(helixManager.get());
+      if (leaderUrl == null) {
+        return;
+      }
+      RestLiServiceException exception = new RestLiServiceException(HttpStatus.S_400_BAD_REQUEST, "Request must be sent to leader node at URL " + leaderUrl);
+      exception.setErrorDetails(new DataMap(ImmutableMap.of(ServiceConfigKeys.LEADER_URL, leaderUrl)));
+      throw exception;
+    }
+  }
+
+  /**
+   * Build helix instance name by getting {@link org.apache.gobblin.service.ServiceConfigKeys#HELIX_INSTANCE_NAME_KEY}
+   * and appending the host, port, and service name with a separator
+   */
+  public static String buildHelixInstanceName(Config config, String defaultInstanceName) {
+    String helixInstanceName = ConfigUtils.getString(config, ServiceConfigKeys.HELIX_INSTANCE_NAME_KEY, defaultInstanceName);
+
+    String url = "";
+    try {
+      url = HELIX_INSTANCE_NAME_SEPARATOR + "https://" + InetAddress.getLocalHost().getHostName() + ":" + ConfigUtils.getString(config, ServiceConfigKeys.SERVICE_PORT, "")

Review comment:
       Added a config for this and default to https://




----------------------------------------------------------------
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] [incubator-gobblin] sv2000 commented on a change in pull request #3124: [GOBBLIN-1061] Add option to only accept requests to leader node and redirect requests in the client

Posted by GitBox <gi...@apache.org>.
sv2000 commented on a change in pull request #3124:
URL: https://github.com/apache/incubator-gobblin/pull/3124#discussion_r503607306



##########
File path: gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-client/src/main/java/org/apache/gobblin/service/FlowConfigClient.java
##########
@@ -104,10 +104,8 @@ public void createFlowConfig(FlowConfig flowConfig)
 
     CreateIdRequest<ComplexResourceKey<FlowId, EmptyRecord>, FlowConfig> request =
         _flowconfigsRequestBuilders.create().input(flowConfig).build();
-    ResponseFuture<IdResponse<ComplexResourceKey<FlowId, EmptyRecord>>> flowConfigResponseFuture =
-        _restClient.get().sendRequest(request);
 
-    flowConfigResponseFuture.getResponse();
+    FlowClientUtils.sendRequestWithRetry(_restClient.get(), request, FlowconfigsRequestBuilders.getPrimaryResource());

Review comment:
       We should let FlowConfigClient untouched, since it is already deprecated. We should in fact remove this code soon.

##########
File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/HelixLeaderUtils.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtime.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.LiveInstance;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.linkedin.data.DataMap;
+import com.linkedin.restli.common.HttpStatus;
+import com.linkedin.restli.server.RestLiServiceException;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Utils for storing/parsing helix URL in the helix instance name
+ */
+@Slf4j
+public class HelixLeaderUtils {
+  public static String HELIX_INSTANCE_NAME_SEPARATOR = "@";
+
+  /**
+   */
+  private static String getUrlFromHelixInstanceName(String helixInstanceName) {
+    if (!helixInstanceName.contains(HELIX_INSTANCE_NAME_SEPARATOR)) {
+      return null;
+    } else {
+      return helixInstanceName.substring(helixInstanceName.indexOf(HELIX_INSTANCE_NAME_SEPARATOR) + 1);
+    }
+  }
+
+  private static String getLeaderUrl(HelixManager helixManager) {
+    PropertyKey key = helixManager.getHelixDataAccessor().keyBuilder().controllerLeader();
+    LiveInstance leader = helixManager.getHelixDataAccessor().getProperty(key);
+    return getUrlFromHelixInstanceName(leader.getInstanceName());
+  }
+
+  /**
+   * If this host is not the leader, throw a {@link RestLiServiceException}, and include the URL of the leader host in
+   * the message and in the errorDetails under the key {@link ServiceConfigKeys#LEADER_URL}.
+   */
+  public static void throwErrorIfNotLeader(Optional<HelixManager> helixManager)  {
+    if (helixManager.isPresent() && !helixManager.get().isLeader()) {
+      String leaderUrl = getLeaderUrl(helixManager.get());
+      if (leaderUrl == null) {
+        return;
+      }
+      RestLiServiceException exception = new RestLiServiceException(HttpStatus.S_400_BAD_REQUEST, "Request must be sent to leader node at URL " + leaderUrl);
+      exception.setErrorDetails(new DataMap(ImmutableMap.of(ServiceConfigKeys.LEADER_URL, leaderUrl)));
+      throw exception;
+    }
+  }
+
+  /**
+   * Build helix instance name by getting {@link org.apache.gobblin.service.ServiceConfigKeys#HELIX_INSTANCE_NAME_KEY}
+   * and appending the host, port, and service name with a separator
+   */
+  public static String buildHelixInstanceName(Config config, String defaultInstanceName) {
+    String helixInstanceName = ConfigUtils.getString(config, ServiceConfigKeys.HELIX_INSTANCE_NAME_KEY, defaultInstanceName);
+
+    String url = "";
+    try {
+      url = HELIX_INSTANCE_NAME_SEPARATOR + "https://" + InetAddress.getLocalHost().getHostName() + ":" + ConfigUtils.getString(config, ServiceConfigKeys.SERVICE_PORT, "")

Review comment:
       We should probably thrown an error if SERVICE_PORT is missing. 

##########
File path: gobblin-runtime/src/main/java/org/apache/gobblin/runtime/util/HelixLeaderUtils.java
##########
@@ -0,0 +1,96 @@
+/*
+ * 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.runtime.util;
+
+import java.net.InetAddress;
+import java.net.UnknownHostException;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.PropertyKey;
+import org.apache.helix.model.LiveInstance;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableMap;
+import com.linkedin.data.DataMap;
+import com.linkedin.restli.common.HttpStatus;
+import com.linkedin.restli.server.RestLiServiceException;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.service.ServiceConfigKeys;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * Utils for storing/parsing helix URL in the helix instance name
+ */
+@Slf4j
+public class HelixLeaderUtils {

Review comment:
       Can this be moved into gobblin-service?

##########
File path: gobblin-runtime/build.gradle
##########
@@ -66,6 +66,7 @@ dependencies {
   compile externalDependency.gson
   compile externalDependency.guava
   compile externalDependency.guice
+  compile externalDependency.helix

Review comment:
       Hmm. I would avoid Helix dependency in gobblin-runtime. 

##########
File path: gobblin-runtime/build.gradle
##########
@@ -66,6 +66,7 @@ dependencies {
   compile externalDependency.gson
   compile externalDependency.guava
   compile externalDependency.guice
+  compile externalDependency.helix

Review comment:
       Can we move FlowStatusGenerator to gobblin-service module?




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