You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gobblin.apache.org by GitBox <gi...@apache.org> on 2021/11/16 19:35:06 UTC

[GitHub] [gobblin] arjun4084346 opened a new pull request #3427: make a copy of helix manager, so that each job can keep using it with…

arjun4084346 opened a new pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427


   …out worrying about disconnect
   
   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
   - [ ] 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-XXX
   
   
   ### Description
   - [ ] Here are some details about my PR, including screenshots (if applicable):
   
   
   ### Tests
   - [ ] My PR adds the following unit tests __OR__ does not need testing for this extremely good reason:
   
   
   ### Commits
   - [ ] 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.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] phet commented on a change in pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
phet commented on a change in pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#discussion_r750676638



##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -161,6 +170,10 @@ public Void call() throws JobException {
     }
 
     try {
+      this.jobHelixManager.connect();
+      if (this.taskDriverHelixManager.isPresent()) {
+        this.taskDriverHelixManager.get().connect();
+      }

Review comment:
       again... don't these belong prior to and outside the `try` (technically even w/ each its own separate, nested `try` block)?

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobTask.java
##########
@@ -203,6 +209,7 @@ public TaskResult run() {
       return new TaskResult(TaskResult.Status.FAILED, "Exception occurred for job " + planningJobId + ":" + ExceptionUtils
           .getFullStackTrace(e));
     } finally {
+      this.jobHelixManager.disconnect();

Review comment:
       formatting is a bit hard to discern (in the PR viewer), but is this the `finally` of the `try` block where the `.connect()` call is the first statement?  if so, shouldn't we either move the `.connect()` directly prior to the `try`, or else begin another `try` right after the `connect()`?

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -126,8 +126,17 @@ public HelixRetriggeringJobCallable(
     this.planningJobLauncherMetrics = planningJobLauncherMetrics;
     this.helixMetrics = helixMetrics;
     this.appWorkDir = appWorkDir;
-    this.jobHelixManager = jobHelixManager;
-    this.taskDriverHelixManager = taskDriverHelixManager;
+
+    // make a copy of helix managers, so that each job can keep using it without worrying about disconnect from the GobblinClusterManager where they are created
+    this.jobHelixManager = GobblinHelixMultiManager.buildHelixManager(ConfigUtils.propertiesToConfig(sysProps),
+        sysProps.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY),
+        GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY, jobHelixManager.getInstanceType());
+
+    this.taskDriverHelixManager = taskDriverHelixManager.map(
+        helixManager -> GobblinHelixMultiManager.buildHelixManager(ConfigUtils.propertiesToConfig(sysProps),
+            sysProps.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY),
+            GobblinClusterConfigurationKeys.TASK_DRIVER_CLUSTER_NAME_KEY, helixManager.getInstanceType()));
+

Review comment:
       when I saw this above, I was wondering whether it shouldn't be abstracted into a utility... now seeing it again (twice!), I really think so.  any impediment to doing so?

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -172,7 +185,10 @@ public Void call() throws JobException {
       if (deleteJobWhenException) {
         deleteJobSpec();
       }
-      throw e;
+      throw new JobException(e.getMessage());
+    } finally {
+      this.jobHelixManager.disconnect();
+      this.taskDriverHelixManager.ifPresent(HelixManager::disconnect);

Review comment:
       I believe these belong in separate `try ... finally` blocks (and, per nesting, `disconnect()` (i.e. closing/finalization) should be in *reverse* order of `connect()` (initialization)




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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5f00218) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `0.06%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   43.35%   +0.06%     
   - Complexity     2017     2026       +9     
   ============================================
     Files           402      404       +2     
     Lines         17325    17343      +18     
     Branches       2121     2119       -2     
   ============================================
   + Hits           7500     7519      +19     
     Misses         8987     8987              
   + Partials        838      837       -1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | `67.04% <ø> (ø)` | |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `63.46% <25.00%> (ø)` | |
   | [...he/gobblin/cluster/GobblinHelixManagerFactory.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TWFuYWdlckZhY3RvcnkuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | `58.87% <66.66%> (+0.93%)` | :arrow_up: |
   | [...ache/gobblin/cluster/GobblinHelixMultiManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TXVsdGlNYW5hZ2VyLmphdmE=) | `52.87% <83.33%> (ø)` | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | `61.29% <87.50%> (+1.15%)` | :arrow_up: |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | `60.86% <100.00%> (+0.86%)` | :arrow_up: |
   | [.../apache/gobblin/cluster/GobblinZkHelixManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblprSGVsaXhNYW5hZ2VyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `39.39% <0.00%> (-6.07%)` | :arrow_down: |
   | [.../gobblin/iceberg/writer/IcebergMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0ljZWJlcmdNZXRhZGF0YVdyaXRlci5qYXZh) | `72.18% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [3 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...5f00218](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] phet commented on a change in pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
phet commented on a change in pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#discussion_r751685140



##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -263,7 +275,7 @@ private void runJobExecutionLauncher() throws JobException {
       jobLock.lock();
 
       try {
-        if (planningJobIdFromStore.isPresent() && !canRun(planningJobIdFromStore.get(), planningJobManager)) {
+        if (planningJobIdFromStore.isPresent() && !canRun(planningJobIdFromStore.get(), planningJobHelixManager)) {

Review comment:
       aah, darn, yes, frustrating...




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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] phet commented on a change in pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
phet commented on a change in pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#discussion_r754553904



##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinZkHelixManager.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.cluster;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.helix.InstanceType;
+import org.apache.helix.manager.zk.ZKHelixManager;
+
+import lombok.extern.slf4j.Slf4j;
+
+
+/**
+ * A {@link ZKHelixManager} which keeps a reference count of users.
+ * Every user should call connect and disconnect to increase and decrease the count.
+ * Calls to connect and disconnect to the underlying ZKHelixManager are made only for the first and last usage respectively.
+ */
+@Slf4j
+public class GobblinZkHelixManager extends ZKHelixManager {

Review comment:
       I'd encourage a more specific name, tracking to how it works, such as `GobblinReferenceCountingZkHelixManager`

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixMultiManager.java
##########
@@ -160,30 +159,28 @@ protected void addLeadershipChangeAwareComponent (LeadershipChangeAwareComponent
   /**
    * Build the {@link HelixManager} for the Application Master.
    */
-  protected static HelixManager buildHelixManager(Config config, String zkConnectionString, String clusterName, InstanceType type) {
+  protected static HelixManager buildHelixManager(Config config, String clusterName, InstanceType type) {
+    Preconditions.checkArgument(config.hasPath(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY));
+    String zkConnectionString = config.getString(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY);
+    log.info("Using ZooKeeper connection string: " + zkConnectionString);
+
     String helixInstanceName = ConfigUtils.getString(config, GobblinClusterConfigurationKeys.HELIX_INSTANCE_NAME_KEY,
         GobblinClusterManager.class.getSimpleName());
-    return HelixManagerFactory.getZKHelixManager(
+    return GobblinHelixManagerFactory.getZKHelixManager(

Review comment:
       ok... I see how this has been used, so apparently not written to a common interface... in that case, probably not worth big changes now.

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinZkHelixManager.java
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.cluster;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.helix.InstanceType;
+import org.apache.helix.manager.zk.ZKHelixManager;
+
+import lombok.extern.slf4j.Slf4j;
+
+
+/**
+ * A {@link ZKHelixManager} which keeps a reference count of users.
+ * Every user should call connect and disconnect to increase and decrease the count.
+ * Calls to connect and disconnect to the underlying ZKHelixManager are made only for the first and last usage respectively.
+ */
+@Slf4j
+public class GobblinZkHelixManager extends ZKHelixManager {
+  final AtomicInteger usageCount = new AtomicInteger(0);

Review comment:
       `private`?

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixManagerFactory.java
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.cluster;
+
+import org.apache.helix.HelixManager;
+import org.apache.helix.InstanceType;
+
+
+public class GobblinHelixManagerFactory {
+
+  public static HelixManager getZKHelixManager(String clusterName, String instanceName,
+      InstanceType type, String zkAddr) {
+    return new GobblinZkHelixManager(clusterName, instanceName, type, zkAddr);

Review comment:
       two things:
   first, surprised there's no interface declaration, as is typical so different implementions of the factory could be used interchangeably.
   second, not a big deal, but the method name and return type names usually match, whereas the specific instance type (created) might match the class name (granted, more considerations when a common interface is being implemented).

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/GobblinHelixJobTask.java
##########
@@ -203,6 +209,7 @@ public TaskResult run() {
       return new TaskResult(TaskResult.Status.FAILED, "Exception occurred for job " + planningJobId + ":" + ExceptionUtils
           .getFullStackTrace(e));
     } finally {
+      this.jobHelixManager.disconnect();

Review comment:
       still wondering on this... doesn't the `.connect()` belong prior to the start of the `try` block?




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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] arjun4084346 commented on a change in pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#discussion_r750703212



##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -126,8 +126,17 @@ public HelixRetriggeringJobCallable(
     this.planningJobLauncherMetrics = planningJobLauncherMetrics;
     this.helixMetrics = helixMetrics;
     this.appWorkDir = appWorkDir;
-    this.jobHelixManager = jobHelixManager;
-    this.taskDriverHelixManager = taskDriverHelixManager;
+
+    // make a copy of helix managers, so that each job can keep using it without worrying about disconnect from the GobblinClusterManager where they are created
+    this.jobHelixManager = GobblinHelixMultiManager.buildHelixManager(ConfigUtils.propertiesToConfig(sysProps),
+        sysProps.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY),
+        GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY, jobHelixManager.getInstanceType());
+
+    this.taskDriverHelixManager = taskDriverHelixManager.map(
+        helixManager -> GobblinHelixMultiManager.buildHelixManager(ConfigUtils.propertiesToConfig(sysProps),
+            sysProps.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY),
+            GobblinClusterConfigurationKeys.TASK_DRIVER_CLUSTER_NAME_KEY, helixManager.getInstanceType()));
+

Review comment:
       The thing I considered is that I was trying to copy a HelixManager, not a ZKHelixManager. That was all the reasoing behind using the utility method.




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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (00ebf69) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `5.83%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   49.12%   +5.83%     
   - Complexity     2017     8862    +6845     
   ============================================
     Files           402     1692    +1290     
     Lines         17325    65313   +47988     
     Branches       2121     7482    +5361     
   ============================================
   + Hits           7500    32082   +24582     
   - Misses         8987    30208   +21221     
   - Partials        838     3023    +2185     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | `67.04% <ø> (ø)` | |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `63.46% <25.00%> (ø)` | |
   | [...he/gobblin/cluster/GobblinHelixManagerFactory.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TWFuYWdlckZhY3RvcnkuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | `58.87% <66.66%> (+0.93%)` | :arrow_up: |
   | [...ache/gobblin/cluster/GobblinHelixMultiManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TXVsdGlNYW5hZ2VyLmphdmE=) | `52.87% <83.33%> (ø)` | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | `61.29% <87.50%> (+1.15%)` | :arrow_up: |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | `60.86% <100.00%> (+0.86%)` | :arrow_up: |
   | [.../apache/gobblin/cluster/GobblinZkHelixManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblprSGVsaXhNYW5hZ2VyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `39.39% <0.00%> (-6.07%)` | :arrow_down: |
   | [.../gobblin/iceberg/writer/IcebergMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0ljZWJlcmdNZXRhZGF0YVdyaXRlci5qYXZh) | `72.18% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [1291 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...00ebf69](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (00ebf69) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `3.45%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   46.74%   +3.45%     
   - Complexity     2017     3193    +1176     
   ============================================
     Files           402      657     +255     
     Lines         17325    25866    +8541     
     Branches       2121     3065     +944     
   ============================================
   + Hits           7500    12090    +4590     
   - Misses         8987    12481    +3494     
   - Partials        838     1295     +457     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | `67.04% <ø> (ø)` | |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `63.46% <25.00%> (ø)` | |
   | [...he/gobblin/cluster/GobblinHelixManagerFactory.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TWFuYWdlckZhY3RvcnkuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | `58.87% <66.66%> (+0.93%)` | :arrow_up: |
   | [...ache/gobblin/cluster/GobblinHelixMultiManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TXVsdGlNYW5hZ2VyLmphdmE=) | `52.87% <83.33%> (ø)` | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | `61.29% <87.50%> (+1.15%)` | :arrow_up: |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | `60.86% <100.00%> (+0.86%)` | :arrow_up: |
   | [.../apache/gobblin/cluster/GobblinZkHelixManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblprSGVsaXhNYW5hZ2VyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `39.39% <0.00%> (-6.07%)` | :arrow_down: |
   | [.../gobblin/iceberg/writer/IcebergMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0ljZWJlcmdNZXRhZGF0YVdyaXRlci5qYXZh) | `72.18% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [256 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...00ebf69](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (00ebf69) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `3.28%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   46.57%   +3.28%     
   - Complexity     2017    10326    +8309     
   ============================================
     Files           402     2073    +1671     
     Lines         17325    80854   +63529     
     Branches       2121     9015    +6894     
   ============================================
   + Hits           7500    37661   +30161     
   - Misses         8987    39717   +30730     
   - Partials        838     3476    +2638     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | `67.04% <ø> (ø)` | |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `63.46% <25.00%> (ø)` | |
   | [...he/gobblin/cluster/GobblinHelixManagerFactory.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TWFuYWdlckZhY3RvcnkuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | `58.87% <66.66%> (+0.93%)` | :arrow_up: |
   | [...ache/gobblin/cluster/GobblinHelixMultiManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TXVsdGlNYW5hZ2VyLmphdmE=) | `52.87% <83.33%> (ø)` | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | `61.29% <87.50%> (+1.15%)` | :arrow_up: |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | `60.86% <100.00%> (+0.86%)` | :arrow_up: |
   | [.../apache/gobblin/cluster/GobblinZkHelixManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblprSGVsaXhNYW5hZ2VyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `39.39% <0.00%> (-6.07%)` | :arrow_down: |
   | [.../gobblin/iceberg/writer/IcebergMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0ljZWJlcmdNZXRhZGF0YVdyaXRlci5qYXZh) | `72.18% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [1672 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...00ebf69](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5f00218) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `3.45%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   46.74%   +3.45%     
   - Complexity     2017     3194    +1177     
   ============================================
     Files           402      657     +255     
     Lines         17325    25866    +8541     
     Branches       2121     3065     +944     
   ============================================
   + Hits           7500    12092    +4592     
   - Misses         8987    12480    +3493     
   - Partials        838     1294     +456     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | `67.04% <ø> (ø)` | |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `63.46% <25.00%> (ø)` | |
   | [...he/gobblin/cluster/GobblinHelixManagerFactory.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TWFuYWdlckZhY3RvcnkuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | `58.87% <66.66%> (+0.93%)` | :arrow_up: |
   | [...ache/gobblin/cluster/GobblinHelixMultiManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TXVsdGlNYW5hZ2VyLmphdmE=) | `52.87% <83.33%> (ø)` | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | `61.29% <87.50%> (+1.15%)` | :arrow_up: |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | `60.86% <100.00%> (+0.86%)` | :arrow_up: |
   | [.../apache/gobblin/cluster/GobblinZkHelixManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblprSGVsaXhNYW5hZ2VyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `39.39% <0.00%> (-6.07%)` | :arrow_down: |
   | [.../gobblin/iceberg/writer/IcebergMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0ljZWJlcmdNZXRhZGF0YVdyaXRlci5qYXZh) | `72.18% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [256 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...5f00218](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5f00218) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `3.28%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   46.57%   +3.28%     
   - Complexity     2017    10326    +8309     
   ============================================
     Files           402     2073    +1671     
     Lines         17325    80854   +63529     
     Branches       2121     9015    +6894     
   ============================================
   + Hits           7500    37657   +30157     
   - Misses         8987    39721   +30734     
   - Partials        838     3476    +2638     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | `67.04% <ø> (ø)` | |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `63.46% <25.00%> (ø)` | |
   | [...he/gobblin/cluster/GobblinHelixManagerFactory.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TWFuYWdlckZhY3RvcnkuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | `58.87% <66.66%> (+0.93%)` | :arrow_up: |
   | [...ache/gobblin/cluster/GobblinHelixMultiManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TXVsdGlNYW5hZ2VyLmphdmE=) | `52.87% <83.33%> (ø)` | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | `61.29% <87.50%> (+1.15%)` | :arrow_up: |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | `60.86% <100.00%> (+0.86%)` | :arrow_up: |
   | [.../apache/gobblin/cluster/GobblinZkHelixManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblprSGVsaXhNYW5hZ2VyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `39.39% <0.00%> (-6.07%)` | :arrow_down: |
   | [.../gobblin/iceberg/writer/IcebergMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0ljZWJlcmdNZXRhZGF0YVdyaXRlci5qYXZh) | `72.18% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [1672 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...5f00218](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] arjun4084346 commented on a change in pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#discussion_r750797627



##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -263,7 +275,7 @@ private void runJobExecutionLauncher() throws JobException {
       jobLock.lock();
 
       try {
-        if (planningJobIdFromStore.isPresent() && !canRun(planningJobIdFromStore.get(), planningJobManager)) {
+        if (planningJobIdFromStore.isPresent() && !canRun(planningJobIdFromStore.get(), planningJobHelixManager)) {

Review comment:
       Actually I tried that, here and at other place. But because the map Function does not expect to throw an exception, it does not end up as clean as this.




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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ef825ce) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `4.17%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   47.46%   +4.17%     
   - Complexity     2017     8302    +6285     
   ============================================
     Files           402     1669    +1267     
     Lines         17325    63512   +46187     
     Branches       2121     6896    +4775     
   ============================================
   + Hits           7500    30143   +22643     
   - Misses         8987    30731   +21744     
   - Partials        838     2638    +1800     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | | |
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | | |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | | |
   | [...bblin/iceberg/writer/GobblinMetadataException.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0dvYmJsaW5NZXRhZGF0YUV4Y2VwdGlvbi5qYXZh) | | |
   | [.../org/apache/gobblin/util/dataset/DatasetUtils.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi11dGlsaXR5L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3V0aWwvZGF0YXNldC9EYXRhc2V0VXRpbHMuamF2YQ==) | | |
   | [.../org/apache/gobblin/util/io/CloseableHttpConn.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi11dGlsaXR5L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3V0aWwvaW8vQ2xvc2VhYmxlSHR0cENvbm4uamF2YQ==) | | |
   | [...in/java/org/apache/gobblin/util/AvroFlattener.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi11dGlsaXR5L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3V0aWwvQXZyb0ZsYXR0ZW5lci5qYXZh) | | |
   | [...org/apache/gobblin/util/test/HelloWorldSource.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi11dGlsaXR5L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3V0aWwvdGVzdC9IZWxsb1dvcmxkU291cmNlLmphdmE=) | | |
   | [...bblin/util/request\_allocation/GreedyAllocator.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi11dGlsaXR5L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3V0aWwvcmVxdWVzdF9hbGxvY2F0aW9uL0dyZWVkeUFsbG9jYXRvci5qYXZh) | | |
   | ... and [2061 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...ef825ce](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter commented on pull request #3427: make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (ef825ce) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `7.91%`.
   > The diff coverage is `n/a`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   51.20%   +7.91%     
   - Complexity     2017     6838    +4821     
   ============================================
     Files           402     1288     +886     
     Lines         17325    47971   +30646     
     Branches       2121     5363    +3242     
   ============================================
   + Hits           7500    24564   +17064     
   - Misses         8987    21222   +12235     
   - Partials        838     2185    +1347     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | | |
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | | |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | | |
   | [.../org/apache/gobblin/util/limiter/MultiLimiter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi11dGlsaXR5L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3V0aWwvbGltaXRlci9NdWx0aUxpbWl0ZXIuamF2YQ==) | | |
   | [...gobblin/cluster/ClusterEventMetadataGenerator.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvQ2x1c3RlckV2ZW50TWV0YWRhdGFHZW5lcmF0b3IuamF2YQ==) | | |
   | [.../apache/gobblin/cluster/GobblinClusterManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkNsdXN0ZXJNYW5hZ2VyLmphdmE=) | | |
   | [...blin/audit/values/auditor/ValueAuditGenerator.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1hdWRpdC9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvZ29iYmxpbi9hdWRpdC92YWx1ZXMvYXVkaXRvci9WYWx1ZUF1ZGl0R2VuZXJhdG9yLmphdmE=) | | |
   | [.../apache/gobblin/metastore/DatasetStoreDataset.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1tZXRhc3RvcmUvc3JjL21haW4vamF2YS9vcmcvYXBhY2hlL2dvYmJsaW4vbWV0YXN0b3JlL0RhdGFzZXRTdG9yZURhdGFzZXQuamF2YQ==) | | |
   | [...src/main/java/org/apache/gobblin/util/Sleeper.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi11dGlsaXR5L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL3V0aWwvU2xlZXBlci5qYXZh) | | |
   | ... and [1680 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...ef825ce](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956






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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5f00218) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `5.82%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   49.11%   +5.82%     
   - Complexity     2017     8862    +6845     
   ============================================
     Files           402     1692    +1290     
     Lines         17325    65313   +47988     
     Branches       2121     7482    +5361     
   ============================================
   + Hits           7500    32079   +24579     
   - Misses         8987    30212   +21225     
   - Partials        838     3022    +2184     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | `67.04% <ø> (ø)` | |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `63.46% <25.00%> (ø)` | |
   | [...he/gobblin/cluster/GobblinHelixManagerFactory.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TWFuYWdlckZhY3RvcnkuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | `58.87% <66.66%> (+0.93%)` | :arrow_up: |
   | [...ache/gobblin/cluster/GobblinHelixMultiManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TXVsdGlNYW5hZ2VyLmphdmE=) | `52.87% <83.33%> (ø)` | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | `61.29% <87.50%> (+1.15%)` | :arrow_up: |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | `60.86% <100.00%> (+0.86%)` | :arrow_up: |
   | [.../apache/gobblin/cluster/GobblinZkHelixManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblprSGVsaXhNYW5hZ2VyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `39.39% <0.00%> (-6.07%)` | :arrow_down: |
   | [.../gobblin/iceberg/writer/IcebergMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0ljZWJlcmdNZXRhZGF0YVdyaXRlci5qYXZh) | `72.18% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [1291 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...5f00218](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] arjun4084346 commented on a change in pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#discussion_r750795320



##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -126,8 +126,17 @@ public HelixRetriggeringJobCallable(
     this.planningJobLauncherMetrics = planningJobLauncherMetrics;
     this.helixMetrics = helixMetrics;
     this.appWorkDir = appWorkDir;
-    this.jobHelixManager = jobHelixManager;
-    this.taskDriverHelixManager = taskDriverHelixManager;
+
+    // make a copy of helix managers, so that each job can keep using it without worrying about disconnect from the GobblinClusterManager where they are created
+    this.jobHelixManager = GobblinHelixMultiManager.buildHelixManager(ConfigUtils.propertiesToConfig(sysProps),
+        sysProps.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY),
+        GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY, jobHelixManager.getInstanceType());
+
+    this.taskDriverHelixManager = taskDriverHelixManager.map(
+        helixManager -> GobblinHelixMultiManager.buildHelixManager(ConfigUtils.propertiesToConfig(sysProps),
+            sysProps.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY),
+            GobblinClusterConfigurationKeys.TASK_DRIVER_CLUSTER_NAME_KEY, helixManager.getInstanceType()));
+

Review comment:
       I agree, the existing code did not look good to me as well. will change




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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] phet commented on a change in pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
phet commented on a change in pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#discussion_r750785564



##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -263,7 +275,7 @@ private void runJobExecutionLauncher() throws JobException {
       jobLock.lock();
 
       try {
-        if (planningJobIdFromStore.isPresent() && !canRun(planningJobIdFromStore.get(), planningJobManager)) {
+        if (planningJobIdFromStore.isPresent() && !canRun(planningJobIdFromStore.get(), planningJobHelixManager)) {

Review comment:
       not pressing, but equivalent:
   ```
   if (planningJobIdFromStore.map(id -> !canRun(id, planningJobHelixManager)).getOrElse(false)) {
   ```

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -249,8 +260,9 @@ private void runJobExecutionLauncher() throws JobException {
     String newPlanningId;
     Closer closer = Closer.create();
     try {
-      HelixManager planningJobManager = this.taskDriverHelixManager.isPresent()?
+      HelixManager planningJobHelixManager = this.taskDriverHelixManager.isPresent()?
           this.taskDriverHelixManager.get() : this.jobHelixManager;

Review comment:
       `getOrElse`?
   
   ...actually thought I earlier saw a similar replacement above...

##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -126,8 +126,17 @@ public HelixRetriggeringJobCallable(
     this.planningJobLauncherMetrics = planningJobLauncherMetrics;
     this.helixMetrics = helixMetrics;
     this.appWorkDir = appWorkDir;
-    this.jobHelixManager = jobHelixManager;
-    this.taskDriverHelixManager = taskDriverHelixManager;
+
+    // make a copy of helix managers, so that each job can keep using it without worrying about disconnect from the GobblinClusterManager where they are created
+    this.jobHelixManager = GobblinHelixMultiManager.buildHelixManager(ConfigUtils.propertiesToConfig(sysProps),
+        sysProps.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY),
+        GobblinClusterConfigurationKeys.HELIX_CLUSTER_NAME_KEY, jobHelixManager.getInstanceType());
+
+    this.taskDriverHelixManager = taskDriverHelixManager.map(
+        helixManager -> GobblinHelixMultiManager.buildHelixManager(ConfigUtils.propertiesToConfig(sysProps),
+            sysProps.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY),
+            GobblinClusterConfigurationKeys.TASK_DRIVER_CLUSTER_NAME_KEY, helixManager.getInstanceType()));
+

Review comment:
       yeah, I suppose `GobblinHelixMultiManager.buildHelixManager` is already a utility, probably excessive to abstract too much beyond.  what I noticed is that (before calling) you're pulling a property out of the same `Config` you provide to the call.  usage shows `ZK_CONNECTION_STRING_KEY` to be pretty widely referenced... what about a 3-param overload:
   ```
   protected static HelixManager buildHelixManager(Config config, String clusterName, InstanceType type) {
     return buildHelixManager(config, config.getProperty(GobblinClusterConfigurationKeys.ZK_CONNECTION_STRING_KEY), clusterName, type);
   }
   ```
   ?
     
   
   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.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] arjun4084346 commented on a change in pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
arjun4084346 commented on a change in pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#discussion_r750795802



##########
File path: gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java
##########
@@ -249,8 +260,9 @@ private void runJobExecutionLauncher() throws JobException {
     String newPlanningId;
     Closer closer = Closer.create();
     try {
-      HelixManager planningJobManager = this.taskDriverHelixManager.isPresent()?
+      HelixManager planningJobHelixManager = this.taskDriverHelixManager.isPresent()?
           this.taskDriverHelixManager.get() : this.jobHelixManager;

Review comment:
       Yes, i removed it in this PR. 1st change of 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.

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] ZihanLi58 merged pull request #3427: [GOBBLIN-1575] use reference count in helix manager, so that connect/disconnect are called once and at the right time

Posted by GitBox <gi...@apache.org>.
ZihanLi58 merged pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427


   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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



[GitHub] [gobblin] codecov-commenter edited a comment on pull request #3427: [GOBBLIN-1575] make a copy of helix manager, so that each job can keep using it with…

Posted by GitBox <gi...@apache.org>.
codecov-commenter edited a comment on pull request #3427:
URL: https://github.com/apache/gobblin/pull/3427#issuecomment-970625956


   # [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#3427](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (00ebf69) into [master](https://codecov.io/gh/apache/gobblin/commit/5bc22a5502ddeb810a35b0fb996dd9dfc8c81121?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (5bc22a5) will **increase** coverage by `0.06%`.
   > The diff coverage is `80.00%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/gobblin/pull/3427/graphs/tree.svg?width=650&height=150&src=pr&token=4MgURJ0bGc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   
   ```diff
   @@             Coverage Diff              @@
   ##             master    #3427      +/-   ##
   ============================================
   + Coverage     43.29%   43.35%   +0.06%     
   - Complexity     2017     2026       +9     
   ============================================
     Files           402      404       +2     
     Lines         17325    17343      +18     
     Branches       2121     2119       -2     
   ============================================
   + Hits           7500     7519      +19     
     Misses         8987     8987              
   + Partials        838      837       -1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [...pache/gobblin/cluster/GobblinHelixJobLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iTGF1bmNoZXIuamF2YQ==) | `67.04% <ø> (ø)` | |
   | [.../org/apache/gobblin/cluster/GobblinTaskRunner.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblRhc2tSdW5uZXIuamF2YQ==) | `63.46% <25.00%> (ø)` | |
   | [...he/gobblin/cluster/GobblinHelixManagerFactory.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TWFuYWdlckZhY3RvcnkuamF2YQ==) | `50.00% <50.00%> (ø)` | |
   | [...er/GobblinHelixDistributeJobExecutionLauncher.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4RGlzdHJpYnV0ZUpvYkV4ZWN1dGlvbkxhdW5jaGVyLmphdmE=) | `58.87% <66.66%> (+0.93%)` | :arrow_up: |
   | [...ache/gobblin/cluster/GobblinHelixMultiManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4TXVsdGlNYW5hZ2VyLmphdmE=) | `52.87% <83.33%> (ø)` | |
   | [.../gobblin/cluster/HelixRetriggeringJobCallable.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvSGVsaXhSZXRyaWdnZXJpbmdKb2JDYWxsYWJsZS5qYXZh) | `61.29% <87.50%> (+1.15%)` | :arrow_up: |
   | [...rg/apache/gobblin/cluster/GobblinHelixJobTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpbkhlbGl4Sm9iVGFzay5qYXZh) | `60.86% <100.00%> (+0.86%)` | :arrow_up: |
   | [.../apache/gobblin/cluster/GobblinZkHelixManager.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvR29iYmxpblprSGVsaXhNYW5hZ2VyLmphdmE=) | `100.00% <100.00%> (ø)` | |
   | [.../java/org/apache/gobblin/cluster/SleepingTask.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1jbHVzdGVyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2NsdXN0ZXIvU2xlZXBpbmdUYXNrLmphdmE=) | `39.39% <0.00%> (-6.07%)` | :arrow_down: |
   | [.../gobblin/iceberg/writer/IcebergMetadataWriter.java](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Z29iYmxpbi1pY2ViZXJnL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9nb2JibGluL2ljZWJlcmcvd3JpdGVyL0ljZWJlcmdNZXRhZGF0YVdyaXRlci5qYXZh) | `72.18% <0.00%> (-0.12%)` | :arrow_down: |
   | ... and [3 more](https://codecov.io/gh/apache/gobblin/pull/3427/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [5bc22a5...00ebf69](https://codecov.io/gh/apache/gobblin/pull/3427?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


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

To unsubscribe, e-mail: dev-unsubscribe@gobblin.apache.org

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