You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eagle.apache.org by ji...@apache.org on 2016/11/15 05:33:32 UTC

incubator-eagle git commit: [EAGLE-764] MR history feeder support Health Checks

Repository: incubator-eagle
Updated Branches:
  refs/heads/master 44a4d660b -> c1ce6ea4f


[EAGLE-764] MR history feeder support Health Checks

Author: wujinhu <wu...@126.com>

Closes #652 from wujinhu/EAGLE-764.


Project: http://git-wip-us.apache.org/repos/asf/incubator-eagle/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-eagle/commit/c1ce6ea4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-eagle/tree/c1ce6ea4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-eagle/diff/c1ce6ea4

Branch: refs/heads/master
Commit: c1ce6ea4f62c2f2ed53099e0b0ccb3a130128c30
Parents: 44a4d66
Author: wujinhu <wu...@126.com>
Authored: Tue Nov 15 13:33:25 2016 +0800
Committer: wujinhu <wu...@126.com>
Committed: Tue Nov 15 13:33:25 2016 +0800

----------------------------------------------------------------------
 eagle-core/eagle-app/eagle-app-base/pom.xml     | 10 +++
 .../java/org/apache/eagle/app/Application.java  | 10 +++
 .../app/module/ApplicationGuiceModule.java      |  3 +
 .../service/ApplicationHealthCheckService.java  | 29 +++++++
 .../impl/ApplicationHealthCheckServiceImpl.java | 84 ++++++++++++++++++++
 .../impl/ApplicationManagementServiceImpl.java  | 10 ++-
 .../mr/MRMetricsAggregateContainer.java         |  5 +-
 .../jpm/aggregation/storm/AggregationSpout.java |  2 +-
 .../jpm/mr/history/MRHistoryJobApplication.java |  6 ++
 .../MRHistoryJobApplicationHealthCheck.java     | 83 +++++++++++++++++++
 .../apache/eagle/server/ServerApplication.java  |  5 ++
 11 files changed, 243 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-core/eagle-app/eagle-app-base/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/pom.xml b/eagle-core/eagle-app/eagle-app-base/pom.xml
index b3c9202..56ed4c4 100644
--- a/eagle-core/eagle-app/eagle-app-base/pom.xml
+++ b/eagle-core/eagle-app/eagle-app-base/pom.xml
@@ -102,5 +102,15 @@
             <artifactId>junit</artifactId>
             <scope>provided</scope>
         </dependency>
+        <dependency>
+            <groupId>io.dropwizard</groupId>
+            <artifactId>dropwizard-core</artifactId>
+            <exclusions>
+                <exclusion>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>log4j-over-slf4j</artifactId>
+                </exclusion>
+            </exclusions>
+        </dependency>
     </dependencies>
 </project>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/Application.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/Application.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/Application.java
index d76e468..9a2ea37 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/Application.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/Application.java
@@ -16,6 +16,7 @@
  */
 package org.apache.eagle.app;
 
+import com.codahale.metrics.health.HealthCheck;
 import org.apache.eagle.app.environment.Environment;
 import com.typesafe.config.Config;
 
@@ -54,4 +55,13 @@ public interface Application<
     Class<? extends E> getEnvironmentType();
 
     boolean isExecutable();
+
+    default HealthCheck getAppHealthCheck(Config config) {
+        return new HealthCheck() {
+            @Override
+            protected Result check() throws Exception {
+                return Result.healthy();
+            }
+        };
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java
index d796faa..6c8c310 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/module/ApplicationGuiceModule.java
@@ -17,8 +17,10 @@
 package org.apache.eagle.app.module;
 
 import com.typesafe.config.ConfigFactory;
+import org.apache.eagle.app.service.ApplicationHealthCheckService;
 import org.apache.eagle.app.service.ApplicationManagementService;
 import org.apache.eagle.app.service.ApplicationProviderService;
+import org.apache.eagle.app.service.impl.ApplicationHealthCheckServiceImpl;
 import org.apache.eagle.app.service.impl.ApplicationManagementServiceImpl;
 import org.apache.eagle.app.service.impl.ApplicationProviderServiceImpl;
 import org.apache.eagle.app.service.impl.ApplicationStatusUpdateServiceImpl;
@@ -45,5 +47,6 @@ public class ApplicationGuiceModule extends AbstractModule {
         bind(ApplicationDescService.class).toProvider(Providers.of(appProviderInst));
         bind(ApplicationManagementService.class).to(ApplicationManagementServiceImpl.class).in(Singleton.class);
         bind(ApplicationStatusUpdateService.class).to(ApplicationStatusUpdateServiceImpl.class).in(Singleton.class);
+        bind(ApplicationHealthCheckService.class).to(ApplicationHealthCheckServiceImpl.class).in(Singleton.class);
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationHealthCheckService.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationHealthCheckService.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationHealthCheckService.java
new file mode 100644
index 0000000..8b6affe
--- /dev/null
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/ApplicationHealthCheckService.java
@@ -0,0 +1,29 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.eagle.app.service;
+
+import io.dropwizard.setup.Environment;
+import org.apache.eagle.metadata.model.ApplicationEntity;
+
+public interface ApplicationHealthCheckService {
+    void init(Environment environment);
+
+    void register(ApplicationEntity appEntity);
+
+    void unregister(ApplicationEntity appEntity);
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationHealthCheckServiceImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationHealthCheckServiceImpl.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationHealthCheckServiceImpl.java
new file mode 100644
index 0000000..bf28e93
--- /dev/null
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationHealthCheckServiceImpl.java
@@ -0,0 +1,84 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.eagle.app.service.impl;
+
+import com.codahale.metrics.health.HealthCheck;
+import com.google.inject.Inject;
+import com.typesafe.config.Config;
+import com.typesafe.config.ConfigFactory;
+import io.dropwizard.setup.Environment;
+import org.apache.eagle.app.service.ApplicationHealthCheckService;
+import org.apache.eagle.app.service.ApplicationProviderService;
+import org.apache.eagle.app.spi.ApplicationProvider;
+import org.apache.eagle.metadata.model.ApplicationEntity;
+import org.apache.eagle.metadata.service.ApplicationEntityService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+
+public class ApplicationHealthCheckServiceImpl implements ApplicationHealthCheckService {
+    private static final Logger LOG = LoggerFactory.getLogger(ApplicationHealthCheckServiceImpl.class);
+
+    private final ApplicationProviderService applicationProviderService;
+    private final ApplicationEntityService applicationEntityService;
+    private final Config config;
+    private Environment environment;
+
+    @Inject
+    public ApplicationHealthCheckServiceImpl(ApplicationProviderService applicationProviderService,
+                                             ApplicationEntityService applicationEntityService,
+                                             Config config) {
+        this.applicationProviderService = applicationProviderService;
+        this.applicationEntityService = applicationEntityService;
+        this.config = config;
+    }
+
+    @Override
+    public void init(Environment environment) {
+        this.environment = environment;
+        Collection<ApplicationEntity> applicationEntities = applicationEntityService.findAll();
+        applicationEntities.forEach(this::register);
+    }
+
+    @Override
+    public void register(ApplicationEntity appEntity) {
+        if (environment == null) {
+            LOG.warn("environment is null, can not register");
+            return;
+        }
+        ApplicationProvider<?> appProvider = applicationProviderService.getApplicationProviderByType(appEntity.getDescriptor().getType());
+        HealthCheck applicationHealthCheck = appProvider.getApplication().getAppHealthCheck(
+                ConfigFactory.parseMap(appEntity.getConfiguration())
+                        .withFallback(config)
+                        .withFallback(ConfigFactory.parseMap(appEntity.getContext()))
+        );
+        this.environment.healthChecks().register(appEntity.getAppId(), applicationHealthCheck);
+        LOG.info("successfully register health check for {}", appEntity.getAppId());
+    }
+
+    @Override
+    public void unregister(ApplicationEntity appEntity) {
+        if (environment == null) {
+            LOG.warn("environment is null, can not unregister");
+            return;
+        }
+        this.environment.healthChecks().unregister(appEntity.getAppId());
+        LOG.info("successfully unregister health check for {}", appEntity.getAppId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
index 003f966..96118aa 100644
--- a/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
+++ b/eagle-core/eagle-app/eagle-app-base/src/main/java/org/apache/eagle/app/service/impl/ApplicationManagementServiceImpl.java
@@ -46,6 +46,7 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
     private final ApplicationEntityService applicationEntityService;
     private final IMetadataDao alertMetadataService;
     private final Config config;
+    private final ApplicationHealthCheckService applicationHealthCheckService;
 
     @Inject private Injector currentInjector;
 
@@ -55,12 +56,14 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
         SiteEntityService siteEntityService,
         ApplicationProviderService applicationProviderService,
         ApplicationEntityService applicationEntityService,
-        IMetadataDao alertMetadataService) {
+        IMetadataDao alertMetadataService,
+        ApplicationHealthCheckService applicationHealthCheckService) {
         this.config = config;
         this.siteEntityService = siteEntityService;
         this.applicationProviderService = applicationProviderService;
         this.applicationEntityService = applicationEntityService;
         this.alertMetadataService = alertMetadataService;
+        this.applicationHealthCheckService = applicationHealthCheckService;
     }
 
     @Override
@@ -112,6 +115,8 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
         ApplicationAction applicationAction = new ApplicationAction(applicationProvider.getApplication(), applicationEntity, config, alertMetadataService);
         applicationAction.doInstall();
 
+        applicationHealthCheckService.register(applicationEntity);
+
         // UpdateMetadata
         ApplicationEntity result =  applicationEntityService.create(applicationEntity);
 
@@ -151,6 +156,9 @@ public class ApplicationManagementServiceImpl implements ApplicationManagementSe
                     listener.init(appEntity);
                     listener.afterUninstall();
                 });
+
+                applicationHealthCheckService.unregister(appEntity);
+
                 return applicationEntityService.delete(appEntity);
             } else {
                 throw new ApplicationWrongStatusException("App: " + appEntity.getAppId() + " status is" + currentStatus + ", uninstall operation is not allowed");

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/mr/MRMetricsAggregateContainer.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/mr/MRMetricsAggregateContainer.java b/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/mr/MRMetricsAggregateContainer.java
index 00d0457..45bbcef 100644
--- a/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/mr/MRMetricsAggregateContainer.java
+++ b/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/mr/MRMetricsAggregateContainer.java
@@ -72,8 +72,9 @@ public class MRMetricsAggregateContainer implements MetricsAggregateContainer, S
                 .pageSize(10)
                 .send();
 
-            List<JobProcessTimeStampEntity> entities = response.getObj();
-            return entities.get(0).getCurrentTimeStamp();
+            List<Map<List<String>, List<Double>>> results = response.getObj();
+            long currentProcessTimeStamp = results.get(0).get("value").get(0).longValue();
+            return currentProcessTimeStamp;
         } catch (Exception e) {
             LOG.warn("{}", e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/storm/AggregationSpout.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/storm/AggregationSpout.java b/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/storm/AggregationSpout.java
index 3ee0519..f19c366 100644
--- a/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/storm/AggregationSpout.java
+++ b/eagle-jpm/eagle-jpm-aggregation/src/main/java/org/apache/eagle/jpm/aggregation/storm/AggregationSpout.java
@@ -72,7 +72,7 @@ public class AggregationSpout extends BaseRichSpout {
                 return;
             }
 
-            long currentJobTime = System.currentTimeMillis();//this.jobProcessTime.fetchLatestJobProcessTime();
+            long currentJobTime = this.jobProcessTime.fetchLatestJobProcessTime();
             //1, get last updateTime;
             lastUpdateTime = AggregationTimeManager.instance().readLastFinishTime();
             if (lastUpdateTime == 0L) {

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
index e4e206f..2b4ffee 100644
--- a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplication.java
@@ -17,6 +17,7 @@
 package org.apache.eagle.jpm.mr.history;
 
 import backtype.storm.topology.BoltDeclarer;
+import com.codahale.metrics.health.HealthCheck;
 import org.apache.eagle.app.StormApplication;
 import org.apache.eagle.app.environment.impl.StormEnvironment;
 import org.apache.eagle.app.sink.StormStreamSink;
@@ -76,4 +77,9 @@ public class MRHistoryJobApplication extends StormApplication {
 
         return topologyBuilder.createTopology();
     }
+
+    @Override
+    public HealthCheck getAppHealthCheck(Config config) {
+        return new MRHistoryJobApplicationHealthCheck(config);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationHealthCheck.java
----------------------------------------------------------------------
diff --git a/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationHealthCheck.java b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationHealthCheck.java
new file mode 100644
index 0000000..91cbbd7
--- /dev/null
+++ b/eagle-jpm/eagle-jpm-mr-history/src/main/java/org/apache/eagle/jpm/mr/history/MRHistoryJobApplicationHealthCheck.java
@@ -0,0 +1,83 @@
+/*
+ * 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
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.eagle.jpm.mr.history;
+
+import com.codahale.metrics.health.HealthCheck;
+import com.typesafe.config.Config;
+import org.apache.eagle.jpm.util.Constants;
+import org.apache.eagle.log.entity.GenericServiceAPIResponseEntity;
+import org.apache.eagle.service.client.IEagleServiceClient;
+import org.apache.eagle.service.client.impl.EagleServiceClientImpl;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+public class MRHistoryJobApplicationHealthCheck extends HealthCheck {
+    private static final Logger LOG = LoggerFactory.getLogger(MRHistoryJobApplicationHealthCheck.class);
+
+    private MRHistoryJobConfig mrHistoryJobConfig;
+    private static final long DEFAULT_MAX_DELAY_TIME = 2 * 60 * 60 * 1000L;
+    private static final String MAX_DELAY_TIME_KEY = "application.maxDelayTime";
+
+    public MRHistoryJobApplicationHealthCheck(Config config) {
+        mrHistoryJobConfig = MRHistoryJobConfig.newInstance(config);
+    }
+
+    @Override
+    public Result check() {
+        try {
+            MRHistoryJobConfig.EagleServiceConfig eagleServiceConfig = mrHistoryJobConfig.getEagleServiceConfig();
+            IEagleServiceClient client = new EagleServiceClientImpl(
+                    eagleServiceConfig.eagleServiceHost,
+                    eagleServiceConfig.eagleServicePort,
+                    eagleServiceConfig.username,
+                    eagleServiceConfig.password);
+
+            String query = String.format("%s[@site=\"%s\"]<@site>{max(currentTimeStamp)}",
+                    Constants.JPA_JOB_PROCESS_TIME_STAMP_NAME,
+                    mrHistoryJobConfig.getJobHistoryEndpointConfig().site);
+
+            GenericServiceAPIResponseEntity response = client
+                    .search(query)
+                    .startTime(0L)
+                    .endTime(System.currentTimeMillis())
+                    .pageSize(10)
+                    .send();
+
+            List<Map<List<String>, List<Double>>> results = response.getObj();
+            long currentProcessTimeStamp = results.get(0).get("value").get(0).longValue();
+            long currentTimeStamp = System.currentTimeMillis();
+            long maxDelayTime = DEFAULT_MAX_DELAY_TIME;
+            if (mrHistoryJobConfig.getConfig().hasPath(MAX_DELAY_TIME_KEY)) {
+                maxDelayTime = mrHistoryJobConfig.getConfig().getLong(MAX_DELAY_TIME_KEY);
+            }
+
+            if (currentTimeStamp - currentProcessTimeStamp > maxDelayTime) {
+                String message = String.format("current process time %sms, delay %sms",
+                        currentProcessTimeStamp, currentTimeStamp - currentProcessTimeStamp);
+                return Result.unhealthy(message);
+            } else {
+                return Result.healthy();
+            }
+        } catch (Exception e) {
+            return Result.unhealthy(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/c1ce6ea4/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
----------------------------------------------------------------------
diff --git a/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java b/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
index 8facd15..b9fa243 100644
--- a/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
+++ b/eagle-server/src/main/java/org/apache/eagle/server/ServerApplication.java
@@ -28,6 +28,7 @@ import io.swagger.jaxrs.config.BeanConfig;
 import io.swagger.jaxrs.listing.ApiListingResource;
 import org.apache.eagle.alert.coordinator.CoordinatorListener;
 import org.apache.eagle.alert.resource.SimpleCORSFiler;
+import org.apache.eagle.app.service.ApplicationHealthCheckService;
 import org.apache.eagle.common.Version;
 import org.apache.eagle.log.base.taggedlog.EntityJsonModule;
 import org.apache.eagle.log.base.taggedlog.TaggedLogAPIEntity;
@@ -93,5 +94,9 @@ class ServerApplication extends Application<ServerConfig> {
         ApplicationStatusUpdateService applicationStatusUpdateService = injector.getInstance(ApplicationStatusUpdateService.class);
         Managed updateAppStatusTask = new ApplicationTask(applicationStatusUpdateService);
         environment.lifecycle().manage(updateAppStatusTask);
+
+        //init application health check environment
+        ApplicationHealthCheckService applicationHealthCheckService = injector.getInstance(ApplicationHealthCheckService.class);
+        applicationHealthCheckService.init(environment);
     }
 }
\ No newline at end of file