[ 
https://issues.apache.org/jira/browse/GOBBLIN-1162?focusedWorklogId=438196&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-438196
 ]

ASF GitHub Bot logged work on GOBBLIN-1162:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 28/May/20 07:04
            Start Date: 28/May/20 07:04
    Worklog Time Spent: 10m 
      Work Description: autumnust commented on a change in pull request #3002:
URL: https://github.com/apache/incubator-gobblin/pull/3002#discussion_r431573541



##########
File path: 
gobblin-api/src/main/java/org/apache/gobblin/commit/NoopCommitStep.java
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.commit;
+
+import java.io.IOException;
+
+
+/**
+ * A no-op implementation of {@link CommitStep}.
+ */
+public class NoopCommitStep implements CommitStep {

Review comment:
       Why is this needed? 

##########
File path: 
gobblin-modules/gobblin-kafka-common/src/test/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaIngestionHealthCheckTest.java
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.source.extractor.extract.kafka;
+
+import java.io.IOException;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.mockito.Mockito;
+import org.testng.Assert;
+import org.testng.annotations.BeforeClass;
+import org.testng.annotations.Test;
+
+import com.google.common.eventbus.EventBus;
+import com.google.common.eventbus.Subscribe;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import com.typesafe.config.ConfigValueFactory;
+
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.util.event.ContainerHealthCheckFailureEvent;
+import org.apache.gobblin.util.eventbus.EventBusFactory;
+
+
+public class KafkaIngestionHealthCheckTest {
+  private EventBus eventBus;
+  private CountDownLatch countDownLatch = new CountDownLatch(1);
+
+  @BeforeClass
+  public void setUp() throws IOException {
+    this.eventBus = 
EventBusFactory.get(ContainerHealthCheckFailureEvent.CONTAINER_HEALTH_CHECK_EVENT_BUS_NAME,
+        SharedResourcesBrokerFactory.getImplicitBroker());
+    this.eventBus.register(this);
+  }
+
+  @Subscribe
+  public void 
handleContainerHealthCheckFailureEvent(ContainerHealthCheckFailureEvent event) {
+    this.countDownLatch.countDown();
+  }
+
+  @Test
+  public void testExecute()
+      throws InterruptedException {
+    Config config = 
ConfigFactory.empty().withValue(KafkaIngestionHealthCheck.KAFKA_INGESTION_HEALTH_CHECK_EXPECTED_CONSUMPTION_RATE_MBPS_KEY,
+        ConfigValueFactory.fromAnyRef(5))
+        
.withValue(KafkaIngestionHealthCheck.KAFKA_INGESTION_HEALTH_CHECK_LATENCY_THRESHOLD_MINUTES_KEY,
 ConfigValueFactory.fromAnyRef(5));
+
+    KafkaExtractorStatsTracker extractorStatsTracker = 
Mockito.mock(KafkaExtractorStatsTracker.class);
+    
Mockito.when(extractorStatsTracker.getMaxIngestionLatency(TimeUnit.MINUTES))
+        .thenReturn(6L)

Review comment:
       I just learned from here that `.thenReturn` could be chained ! 

##########
File path: 
gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/source/extractor/extract/kafka/KafkaIngestionHealthCheck.java
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.source.extractor.extract.kafka;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.EvictingQueue;
+import com.google.common.eventbus.EventBus;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.annotation.Alias;
+import org.apache.gobblin.broker.SharedResourcesBrokerFactory;
+import org.apache.gobblin.commit.CommitStep;
+import org.apache.gobblin.util.ConfigUtils;
+import org.apache.gobblin.util.event.ContainerHealthCheckFailureEvent;
+import org.apache.gobblin.util.eventbus.EventBusFactory;
+
+
+@Slf4j
+@Alias(value = "KafkaIngestionHealthCheck")
+public class KafkaIngestionHealthCheck implements CommitStep {
+  public static final String KAFKA_INGESTION_HEALTH_CHECK_PREFIX = 
"gobblin.kafka.healthCheck.";
+  public static final String 
KAFKA_INGESTION_HEALTH_CHECK_SLIDING_WINDOW_SIZE_KEY = 
KAFKA_INGESTION_HEALTH_CHECK_PREFIX + "slidingWindow.size";
+  public static final String 
KAFKA_INGESTION_HEALTH_CHECK_LATENCY_THRESHOLD_MINUTES_KEY = 
KAFKA_INGESTION_HEALTH_CHECK_PREFIX + "ingestionLatency.minutes";
+  public static final String 
KAFKA_INGESTION_HEALTH_CHECK_CONSUMPTION_RATE_DROPOFF_FRACTION_KEY = 
KAFKA_INGESTION_HEALTH_CHECK_PREFIX + "consumptionRate.dropOffFraction";
+  public static final String 
KAFKA_INGESTION_HEALTH_CHECK_EXPECTED_CONSUMPTION_RATE_MBPS_KEY = 
KAFKA_INGESTION_HEALTH_CHECK_PREFIX + "expected.consumptionRateMbps";
+
+  public static final int 
DEFAULT_KAFKA_INGESTION_HEALTH_CHECK_SLIDING_WINDOW_SIZE = 3;
+  public static final long 
DEFAULT_KAFKA_INGESTION_HEALTH_CHECK_LATENCY_THRESHOLD_MINUTES= 15;
+  public static final double 
DEFAULT_KAFKA_INGESTION_HEALTH_CHECK_CONSUMPTION_RATE_DROPOFF_FRACTION = 0.7;
+  public static final double 
DEFAULT_KAFKA_INGESTION_HEALTH_CHECK_EXPECTED_CONSUMPTION_RATE_MBPS = 10.0;
+
+  private final Config config;
+  private final EventBus eventBus;
+  private final KafkaExtractorStatsTracker statsTracker;
+  private final double expectedConsumptionRate;
+  private final double consumptionRateDropOffFraction;
+  private final long ingestionLatencyThresholdMinutes;
+  private final int slidingWindowSize;
+  private final EvictingQueue<Long> ingestionLatencies;
+  private final EvictingQueue<Double> consumptionRateMBps;
+
+  public KafkaIngestionHealthCheck(Config config, KafkaExtractorStatsTracker 
statsTracker) {
+    this.config = config;
+    this.slidingWindowSize = ConfigUtils.getInt(config, 
KAFKA_INGESTION_HEALTH_CHECK_SLIDING_WINDOW_SIZE_KEY, 
DEFAULT_KAFKA_INGESTION_HEALTH_CHECK_SLIDING_WINDOW_SIZE);
+    this.ingestionLatencyThresholdMinutes = ConfigUtils.getLong(config, 
KAFKA_INGESTION_HEALTH_CHECK_LATENCY_THRESHOLD_MINUTES_KEY, 
DEFAULT_KAFKA_INGESTION_HEALTH_CHECK_LATENCY_THRESHOLD_MINUTES);
+    this.consumptionRateDropOffFraction = ConfigUtils.getDouble(config, 
KAFKA_INGESTION_HEALTH_CHECK_CONSUMPTION_RATE_DROPOFF_FRACTION_KEY, 
DEFAULT_KAFKA_INGESTION_HEALTH_CHECK_CONSUMPTION_RATE_DROPOFF_FRACTION);
+    this.expectedConsumptionRate = ConfigUtils.getDouble(config, 
KAFKA_INGESTION_HEALTH_CHECK_EXPECTED_CONSUMPTION_RATE_MBPS_KEY, 
DEFAULT_KAFKA_INGESTION_HEALTH_CHECK_EXPECTED_CONSUMPTION_RATE_MBPS);
+    this.ingestionLatencies = EvictingQueue.create(this.slidingWindowSize);
+    this.consumptionRateMBps = EvictingQueue.create(this.slidingWindowSize);
+    EventBus eventBus;
+    try {
+      eventBus = 
EventBusFactory.get(ContainerHealthCheckFailureEvent.CONTAINER_HEALTH_CHECK_EVENT_BUS_NAME,
+          SharedResourcesBrokerFactory.getImplicitBroker());
+    } catch (IOException e) {
+      log.error("Could not find EventBus instance for container health check", 
e);
+      eventBus = null;
+    }
+    this.eventBus = eventBus;
+    this.statsTracker = statsTracker;
+  }
+
+  /**
+   *
+   * @return true if ingestionLatency in the each of the recent epochs exceeds 
the threshold latency and the latency
+   * is increasing over these epochs.
+   */
+  private boolean checkIngestionLatency() {
+    Long previousLatency = -1L;
+    for (Long ingestionLatency: ingestionLatencies) {
+      if (ingestionLatency < this.ingestionLatencyThresholdMinutes || 
previousLatency > ingestionLatency) {
+        return false;
+      } else {
+        previousLatency = ingestionLatency;
+      }
+    }
+    return true;
+  }
+
+  /**
+   * Determine whether the commit step has been completed.
+   */
+  @Override
+  public boolean isCompleted()

Review comment:
       Why this is always "not completed" while `HelixParticipantCheck` 
actually set this when a check finished. 

##########
File path: 
gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnTaskRunner.java
##########
@@ -209,6 +210,12 @@ public static void main(String[] args) throws Exception {
     } catch (ParseException pe) {
       printUsage(options);
       System.exit(1);
+    } catch (ContainerHealthCheckException e) {
+      // Ideally, we should not be catching this exception, as this is 
indicative of a non-recoverable exception. However,
+      // simply propagating the exception may prevent the container exit due 
to the presence of non-daemon threads present
+      // in the application. Hence, we catch this exception to invoke 
System.exit() which in turn ensures that all non-daemon threads are killed.
+      LOGGER.error("Exception encountered: {}", e);
+      System.exit(1);

Review comment:
       Here we just give up this container by terminate this JVM, is that right 
? Or it just give up the application JVM instead of the owning JVM which 
represents the container itself ? Not very sure about this. 
   
   Do we also need to set non-adherence attribute to avoid new yarn container 
allocated on the same physical node, in the AMRMCallbackHandler? 




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

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


Issue Time Tracking
-------------------

    Worklog Id:     (was: 438196)
    Time Spent: 20m  (was: 10m)

> Provide an option to allow slow containers to commit suicide when unhealthy
> ---------------------------------------------------------------------------
>
>                 Key: GOBBLIN-1162
>                 URL: https://issues.apache.org/jira/browse/GOBBLIN-1162
>             Project: Apache Gobblin
>          Issue Type: Improvement
>          Components: gobblin-cluster
>    Affects Versions: 0.15.0
>            Reporter: Sudarshan Vasudevan
>            Assignee: Hung Tran
>            Priority: Major
>             Fix For: 0.15.0
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> In execution environments such as Gobblin-on-Yarn, where Gobblin workers can 
> be re-assigned when the worker dies or is killed, it is useful to add a mode 
> where each Gobblin task running inside a Gobblin worker can perform 
> application-level health checks and report results of the health checks back 
> to the worker hosting the tasks. The worker on receiving a health check 
> failure event can be configured to exit the JVM. In the case of 
> Gobblin-on-Yarn mode, this will result in the worker getting re-assigned to a 
> different node. The proposed behavior is similar in flavor to speculative 
> execution modes provided in other execution frameworks such as MapReduce. 
> This change also provides an example of such an application-level health 
> check that arises in the case of Kafka ingestion. 
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to