You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@eagle.apache.org by li...@apache.org on 2016/01/07 05:08:45 UTC

[2/2] incubator-eagle git commit: EAGLE-97 Enable GC Log monitoring

EAGLE-97 Enable GC Log monitoring

https://issues.apache.org/jira/browse/EAGLE-97

Author: @sunlibin <li...@apache.org>
Reviewer: @RalphSu <su...@gmail.com>

Closes #50


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

Branch: refs/heads/master
Commit: e087501378ecf68e04e2546a46f718c845d06a70
Parents: dd72ace
Author: sunlibin <ab...@gmail.com>
Authored: Thu Jan 7 12:04:16 2016 +0800
Committer: sunlibin <ab...@gmail.com>
Committed: Thu Jan 7 12:07:11 2016 +0800

----------------------------------------------------------------------
 .../eagle/alert/entity/AlertStreamEntity.java   |   2 +-
 .../apache/eagle/executor/AlertExecutor.java    |  10 +-
 .../src/main/resources/ALERT_DEFAULT.vm         |  13 +-
 .../eagle/alert/cep/TestSiddhiEvaluator.java    |   2 +-
 .../eagle/common/config/EagleConfigHelper.java  |  51 ++++
 eagle-gc/pom.xml                                |  50 ++++
 .../org/apache/eagle/gc/GCLogProcessorMain.java |  42 ++++
 .../org/apache/eagle/gc/common/GCConstants.java |  30 +++
 .../gc/executor/GCLogAnalysorExecutor.java      |  77 ++++++
 .../gc/executor/GCMetricGeneratorExecutor.java  | 107 ++++++++
 .../apache/eagle/gc/model/GCPausedEvent.java    | 249 +++++++++++++++++++
 .../apache/eagle/gc/parser/DateTimeParser.java  |  37 +++
 .../apache/eagle/gc/parser/GCEventParser.java   |  31 +++
 .../eagle/gc/parser/GCEventParserEnum.java      |  46 ++++
 .../java/org/apache/eagle/gc/parser/GCType.java |  26 ++
 .../eagle/gc/parser/OtherLogFormatParser.java   |  39 +++
 .../exception/IgnoredLogFormatException.java    |  39 +++
 .../UnrecognizedLogFormatException.java         |  39 +++
 .../full/ConcurrentModeFailureParser.java       |  62 +++++
 .../gc/parser/full/NormalFullGCParser.java      |  79 ++++++
 .../full/ParaNewPromotionFailureParser.java     | 106 ++++++++
 .../gc/parser/tenured/CMSInitialMarkParser.java |  63 +++++
 .../gc/parser/tenured/CMSRemarkParser.java      |  72 ++++++
 .../eagle/gc/parser/young/ParaNewParser.java    |  77 ++++++
 .../eagle/gc/spout/GCLogDeserializer.java       |  39 +++
 .../apache/eagle/gc/stream/GCStreamBuilder.java |  46 ++++
 eagle-gc/src/main/resources/alert-gc-policy.sh  |  21 ++
 .../main/resources/alert-metadata-create-gc.sh  |  27 ++
 eagle-gc/src/main/resources/application.conf    |  60 +++++
 eagle-gc/src/main/resources/gc-storm.yaml       |  18 ++
 eagle-gc/src/main/resources/log4j.properties    |  39 +++
 .../java/org/apache/eagle/TestGCLogParser.java  | 118 +++++++++
 .../eagle/metric/kafka/KafkaOffsetSpout.java    |   4 +-
 pom.xml                                         |   1 +
 34 files changed, 1702 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-core/eagle-alert/eagle-alert-base/src/main/java/org/apache/eagle/alert/entity/AlertStreamEntity.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert/eagle-alert-base/src/main/java/org/apache/eagle/alert/entity/AlertStreamEntity.java b/eagle-core/eagle-alert/eagle-alert-base/src/main/java/org/apache/eagle/alert/entity/AlertStreamEntity.java
index f0f81c5..f0f2c7b 100644
--- a/eagle-core/eagle-alert/eagle-alert-base/src/main/java/org/apache/eagle/alert/entity/AlertStreamEntity.java
+++ b/eagle-core/eagle-alert/eagle-alert-base/src/main/java/org/apache/eagle/alert/entity/AlertStreamEntity.java
@@ -39,7 +39,7 @@ import org.apache.eagle.log.entity.meta.TimeSeries;
 @Tags({"dataSource", "streamName"})
 public class AlertStreamEntity extends TaggedLogAPIEntity{
 	@Column("a")
-	private String desc;	
+	private String desc;
 
 	public String getDesc() {
 		return desc;

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-core/eagle-alert/eagle-alert-process/src/main/java/org/apache/eagle/executor/AlertExecutor.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert/eagle-alert-process/src/main/java/org/apache/eagle/executor/AlertExecutor.java b/eagle-core/eagle-alert/eagle-alert-process/src/main/java/org/apache/eagle/executor/AlertExecutor.java
index 124633c..44014da 100644
--- a/eagle-core/eagle-alert/eagle-alert-process/src/main/java/org/apache/eagle/executor/AlertExecutor.java
+++ b/eagle-core/eagle-alert/eagle-alert-process/src/main/java/org/apache/eagle/executor/AlertExecutor.java
@@ -118,7 +118,7 @@ public class AlertExecutor extends JavaStormStreamExecutor2<String, AlertAPIEnti
 		this.config = config;
 	}
 	
-	public void initMetricReportor() {
+	private void initMetricReportor() {
 		String host = config.getString(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.EAGLE_SERVICE + "." + EagleConfigConstants.HOST);
 		int port = config.getInt(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.EAGLE_SERVICE + "." + EagleConfigConstants.PORT);
 
@@ -250,11 +250,11 @@ public class AlertExecutor extends JavaStormStreamExecutor2<String, AlertAPIEnti
 		return false;
 	}
 	
-	public long trim(long value, long granularity) {
+	private long trim(long value, long granularity) {
 		return value / granularity * granularity;
 	}
 
-	public void updateCounter(String name, Map<String, String> dimensions, double value) {
+	private void updateCounter(String name, Map<String, String> dimensions, double value) {
 		long current = System.currentTimeMillis();
 		String metricName = MetricKeyCodeDecoder.codeMetricKey(name, dimensions);
 		if (registry.getMetrics().get(metricName) == null) {
@@ -268,11 +268,11 @@ public class AlertExecutor extends JavaStormStreamExecutor2<String, AlertAPIEnti
 		}
 	}
 	
-	public void updateCounter(String name, Map<String, String> dimensions) {
+	private void updateCounter(String name, Map<String, String> dimensions) {
 		updateCounter(name, dimensions, 1.0);
 	}
 	
-	public Map<String, String> getDimensions(String policyId) {
+	protected Map<String, String> getDimensions(String policyId) {
 		if (dimensionsMap.get(policyId) == null) {
 			Map<String, String> newDimensions = new HashMap<String, String>(baseDimensions);
 			newDimensions.put(AlertConstants.POLICY_ID, policyId);

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-core/eagle-alert/eagle-alert-process/src/main/resources/ALERT_DEFAULT.vm
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert/eagle-alert-process/src/main/resources/ALERT_DEFAULT.vm b/eagle-core/eagle-alert/eagle-alert-process/src/main/resources/ALERT_DEFAULT.vm
index 4ceabad..d4571f3 100644
--- a/eagle-core/eagle-alert/eagle-alert-process/src/main/resources/ALERT_DEFAULT.vm
+++ b/eagle-core/eagle-alert/eagle-alert-process/src/main/resources/ALERT_DEFAULT.vm
@@ -124,9 +124,6 @@
 							<td style="padding: 0 0 0 0;" align="left" >
 								<p style="color:#FFFFFF;font-weight: bold; font-size: 24px">Eagle</p>
 							</td>
-							<td style="padding: 0 0 0 0;" align="right">
-								<p style="color:#FFFFFF;font-weight: bold;">DAM Alert</p>
-							</td>
 						</tr>
 					</table>
 				</td>
@@ -139,7 +136,7 @@
 						<tr>
 							<!-- Title -->
 							<td align="center">
-								<h1>Malicious Data Operation Detected</h1>
+								<h1>$elem["dataSource"] Alert Detected</h1>
 							</td>
 						</tr>
 						<tr>
@@ -214,9 +211,6 @@
 									</tr>
 									<tr>
 										<th>
-											<p>Alert Type</p>
-										</th>
-										<th>
 											<p>Policy Name</p>
 										</th>
 										<th>
@@ -225,9 +219,6 @@
 									</tr>
 									<tr>
 										<td>
-											<p>DAM Alert</p>
-										</td>									
-										<td>
 											<p>$elem["policyId"]</p>
 										</td>
 										<td>
@@ -258,7 +249,7 @@
 						<tr>
 							<!-- Possible Root Causes Content -->
 							<td class="panel" valign="top" style="background: #F4F4F4; border: 1px solid #AAAAAA; padding: 10px 10px 12px 10px;">
-								<p> Malicious data operation found, please check.</p>
+								<p> $elem["dataSource"] alert found, please check.</p>
 							</td>
 						</tr>
 						<tr>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-core/eagle-alert/eagle-alert-process/src/test/java/org/apache/eagle/alert/cep/TestSiddhiEvaluator.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-alert/eagle-alert-process/src/test/java/org/apache/eagle/alert/cep/TestSiddhiEvaluator.java b/eagle-core/eagle-alert/eagle-alert-process/src/test/java/org/apache/eagle/alert/cep/TestSiddhiEvaluator.java
index c1b4185..521790a 100644
--- a/eagle-core/eagle-alert/eagle-alert-process/src/test/java/org/apache/eagle/alert/cep/TestSiddhiEvaluator.java
+++ b/eagle-core/eagle-alert/eagle-alert-process/src/test/java/org/apache/eagle/alert/cep/TestSiddhiEvaluator.java
@@ -104,7 +104,7 @@ public class TestSiddhiEvaluator {
 
 		context.alertExecutor = new AlertExecutor("alertExecutorId", null, 3, 1, alertDao, new String[]{"hdfsAuditLogEventStream"}) {
 			@Override
-			public Map<String, String> getDimensions(String policyId) {
+			protected Map<String, String> getDimensions(String policyId) {
 				return new HashMap<String, String>();
 			}
 		};

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigHelper.java
----------------------------------------------------------------------
diff --git a/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigHelper.java b/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigHelper.java
new file mode 100644
index 0000000..b8ac75e
--- /dev/null
+++ b/eagle-core/eagle-query/eagle-common/src/main/java/org/apache/eagle/common/config/EagleConfigHelper.java
@@ -0,0 +1,51 @@
+/*
+ *
+ *    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.eagle.common.config;
+
+import com.typesafe.config.Config;
+
+public class EagleConfigHelper {
+
+    public static String getServiceHost(Config config) {
+        return config.getString(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.EAGLE_SERVICE + "." + EagleConfigConstants.HOST);
+    }
+
+    public static int getServicePort(Config config) {
+        return config.getInt(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.EAGLE_SERVICE + "." + EagleConfigConstants.PORT);
+    }
+
+    public static String getServiceUser(Config config) {
+        return config.hasPath(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.EAGLE_SERVICE + "." + EagleConfigConstants.USERNAME) ?
+               config.getString(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.EAGLE_SERVICE + "." + EagleConfigConstants.USERNAME) : null;
+    }
+
+    public static String getServicePassword(Config config) {
+        return config.hasPath(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.EAGLE_SERVICE + "." + EagleConfigConstants.PASSWORD) ?
+                config.getString(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.EAGLE_SERVICE + "." + EagleConfigConstants.PASSWORD) : null;
+    }
+
+    public static String getSite(Config config) {
+        return config.getString(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.SITE);
+    }
+
+    public static String getDataSource(Config config) {
+        return config.getString(EagleConfigConstants.EAGLE_PROPS + "." + EagleConfigConstants.DATA_SOURCE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/pom.xml
----------------------------------------------------------------------
diff --git a/eagle-gc/pom.xml b/eagle-gc/pom.xml
new file mode 100644
index 0000000..1535499
--- /dev/null
+++ b/eagle-gc/pom.xml
@@ -0,0 +1,50 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ /*
+  ~  * 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.
+  ~  */
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+		<groupId>eagle</groupId>
+		<artifactId>eagle-parent</artifactId>
+		<version>0.3.0</version>
+      <relativePath>../pom.xml</relativePath>
+  </parent>
+  <artifactId>eagle-gc</artifactId>
+  <name>eagle-gc</name>
+  <description>eagle gc activity monitoring project</description>
+  <packaging>jar</packaging>
+  <dependencies>
+    <dependency>
+      <groupId>eagle</groupId>
+      <artifactId>eagle-metric</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>eagle</groupId>
+      <artifactId>eagle-stream-process-api</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>eagle</groupId>
+      <artifactId>eagle-alert-process</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+  </dependencies>
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/GCLogProcessorMain.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/GCLogProcessorMain.java b/eagle-gc/src/main/java/org/apache/eagle/gc/GCLogProcessorMain.java
new file mode 100644
index 0000000..278a5bc
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/GCLogProcessorMain.java
@@ -0,0 +1,42 @@
+/*
+ *
+ *    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.eagle.gc;
+
+import com.typesafe.config.Config;
+import org.apache.eagle.dataproc.impl.storm.kafka.KafkaSourcedSpoutProvider;
+import org.apache.eagle.datastream.ExecutionEnvironments;
+import org.apache.eagle.datastream.storm.StormExecutionEnvironment;
+import org.apache.eagle.gc.executor.GCLogAnalysorExecutor;
+import org.apache.eagle.gc.executor.GCMetricGeneratorExecutor;
+
+public class GCLogProcessorMain {
+
+    public static void main(String[] args) throws Exception{
+        StormExecutionEnvironment env = ExecutionEnvironments.getStorm(args);
+        Config config = env.getConfig();
+        KafkaSourcedSpoutProvider provider = new KafkaSourcedSpoutProvider();
+        GCLogAnalysorExecutor logAnalysor = new GCLogAnalysorExecutor();
+        env.fromSpout(provider.getSpout(config)).withOutputFields(1).nameAs("kafkaMsgConsumer")
+                                    .flatMap(logAnalysor)
+                                    .flatMap(new GCMetricGeneratorExecutor())
+                                    .alertWithConsumer("NNGCLogStream", "NNGCAlert");
+        env.execute();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/common/GCConstants.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/common/GCConstants.java b/eagle-gc/src/main/java/org/apache/eagle/gc/common/GCConstants.java
new file mode 100644
index 0000000..218596b
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/common/GCConstants.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.eagle.gc.common;
+
+import org.apache.commons.lang.time.DateUtils;
+
+public class GCConstants {
+    public static String GC_PAUSE_TIME_METRIC_NAME = "eagle.namenode.gc.pausetime";
+    public static long GC_PAUSE_TIME_METRIC_GRANULARITY = 2 * DateUtils.MILLIS_PER_MINUTE;
+    public static String GC_YOUNG_MEMORY_METRIC_NAME = "eagle.namenode.gc.memory.young.used";
+    public static String GC_TENURED_MEMORY_METRIC_NAME = "eagle.namenode.gc.memory.tenured.used";
+    public static String GC_TOTAL_MEMORY_METRIC_NAME = "eagle.namenode.gc.memory.total.used";
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/executor/GCLogAnalysorExecutor.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/executor/GCLogAnalysorExecutor.java b/eagle-gc/src/main/java/org/apache/eagle/gc/executor/GCLogAnalysorExecutor.java
new file mode 100644
index 0000000..5eec9ba
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/executor/GCLogAnalysorExecutor.java
@@ -0,0 +1,77 @@
+/*
+ *
+ *    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.eagle.gc.executor;
+
+import com.typesafe.config.Config;
+import org.apache.eagle.datastream.Collector;
+import org.apache.eagle.datastream.JavaStormStreamExecutor2;
+import org.apache.eagle.datastream.Tuple2;
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.stream.GCStreamBuilder;
+import org.apache.eagle.gc.parser.exception.IgnoredLogFormatException;
+import org.apache.eagle.gc.parser.exception.UnrecognizedLogFormatException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.Map;
+
+public class GCLogAnalysorExecutor extends JavaStormStreamExecutor2<String, Map> {
+
+    public final static Logger LOG = LoggerFactory.getLogger(GCLogAnalysorExecutor.class);
+
+    private Config config;
+
+    private long previousLogTime;
+
+    @Override
+    public void prepareConfig(Config config) {
+        this.config = config;
+    }
+
+    @Override
+    public void init() {
+
+    }
+
+    @Override
+    public void flatMap(List<Object> input, Collector<Tuple2<String, Map>> collector) {
+        String log = (String)input.get(0);
+        GCStreamBuilder builder = new GCStreamBuilder();
+        try {
+            GCPausedEvent pauseEvent = builder.build(log);
+            // Because some gc log like concurrent mode failure may miss timestamp info, so we set the previous log's timestamp for it
+            if (pauseEvent.getTimestamp() == 0) {
+                pauseEvent.setTimestamp(previousLogTime);
+            }
+            previousLogTime = pauseEvent.getTimestamp();
+            collector.collect(new Tuple2("GCLog", pauseEvent.toMap()));
+        }
+        catch (IgnoredLogFormatException ex1) {
+            //DO nothing
+        }
+        catch (UnrecognizedLogFormatException ex2) {
+            LOG.warn(ex2.getMessage());
+        }
+        catch (Exception ex3) {
+            LOG.error("Got an exception when parsing log: ", ex3);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/executor/GCMetricGeneratorExecutor.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/executor/GCMetricGeneratorExecutor.java b/eagle-gc/src/main/java/org/apache/eagle/gc/executor/GCMetricGeneratorExecutor.java
new file mode 100644
index 0000000..03f00aa
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/executor/GCMetricGeneratorExecutor.java
@@ -0,0 +1,107 @@
+/*
+ *
+ *    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.eagle.gc.executor;
+
+import com.codahale.metrics.MetricRegistry;
+import com.typesafe.config.Config;
+import org.apache.eagle.common.config.EagleConfigConstants;
+import org.apache.eagle.common.config.EagleConfigHelper;
+import org.apache.eagle.datastream.*;
+import org.apache.eagle.gc.common.GCConstants;
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.metric.reportor.*;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.*;
+
+public class GCMetricGeneratorExecutor extends JavaStormStreamExecutor2<String, Map> {
+
+    public final static Logger LOG = LoggerFactory.getLogger(GCMetricGeneratorExecutor.class);
+    private Config config;
+    private MetricRegistry registry;
+    private String gcPausedTimeMetricName;
+    private String youngHeapUsageMetricName;
+    private String tenuredHeapUsageMetricName;
+    private String totalHeapUsageMetricName;
+    private Map<String, String> dimensions;
+    private List<EagleMetric> metrics = new ArrayList<>();
+
+    private EagleServiceReporterMetricListener listener;
+
+    @Override
+    public void prepareConfig(Config config) {
+        this.config = config;
+    }
+
+    @Override
+    public void init() {
+        registry = new MetricRegistry();
+        String host = EagleConfigHelper.getServiceHost(config);
+        int port = EagleConfigHelper.getServicePort(config);
+        String username = EagleConfigHelper.getServiceUser(config);
+        String password = EagleConfigHelper.getServicePassword(config);
+        listener = new EagleServiceReporterMetricListener(host, port, username, password);
+        dimensions = new HashMap<>();
+        dimensions.put(EagleConfigConstants.SITE, EagleConfigHelper.getSite(config));
+        dimensions.put(EagleConfigConstants.DATA_SOURCE, EagleConfigHelper.getDataSource(config));
+        gcPausedTimeMetricName = MetricKeyCodeDecoder.codeMetricKey(GCConstants.GC_PAUSE_TIME_METRIC_NAME, dimensions);
+    }
+
+    public void registerMetricIfMissing(String metricName, EagleMetric metric) {
+        if (registry.getMetrics().get(metricName) == null) {
+            metric.registerListener(listener);
+            registry.register(metricName, metric);
+        }
+    }
+
+    @Override
+    public void flatMap(List<Object> input, Collector<Tuple2<String, Map>> collector) {
+        Map<String, Object> map = (Map<String, Object>) input.get(1);
+        GCPausedEvent event = new GCPausedEvent(map);
+        // Generate gc paused time metric
+        EagleCounterMetric metric = new EagleCounterMetric(event.getTimestamp(), gcPausedTimeMetricName, event.getPausedGCTimeSec(), GCConstants.GC_PAUSE_TIME_METRIC_GRANULARITY);
+        registerMetricIfMissing(gcPausedTimeMetricName, metric);
+
+        // Generate young heap paused time metric
+        if (event.isYoungAreaGCed()) {
+            youngHeapUsageMetricName = MetricKeyCodeDecoder.codeTSMetricKey(event.getTimestamp(), GCConstants.GC_YOUNG_MEMORY_METRIC_NAME, dimensions);
+            EagleGaugeMetric metric2 = new EagleGaugeMetric(event.getTimestamp(), youngHeapUsageMetricName, event.getYoungUsedHeapK());
+            metrics.add(metric2);
+        }
+
+        // Generate tenured heap paused time metric
+        if (event.isTenuredAreaGCed()) {
+            tenuredHeapUsageMetricName = MetricKeyCodeDecoder.codeTSMetricKey(event.getTimestamp(), GCConstants.GC_TENURED_MEMORY_METRIC_NAME, dimensions);
+            EagleGaugeMetric metric3 = new EagleGaugeMetric(event.getTimestamp(), tenuredHeapUsageMetricName, event.getTenuredUsedHeapK());
+            metrics.add(metric3);
+        }
+
+        // Generate total heap paused time metric
+        if (event.isTotalHeapUsageAvailable()) {
+            totalHeapUsageMetricName = MetricKeyCodeDecoder.codeTSMetricKey(event.getTimestamp(), GCConstants.GC_TOTAL_MEMORY_METRIC_NAME, dimensions);
+            EagleGaugeMetric metric4 = new EagleGaugeMetric(event.getTimestamp(), totalHeapUsageMetricName, event.getUsedTotalHeapK());
+            metrics.add(metric4);
+        }
+        listener.onMetricFlushed(metrics);
+        metrics.clear();
+        collector.collect(new Tuple2(input.get(0), input.get(1)));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/model/GCPausedEvent.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/model/GCPausedEvent.java b/eagle-gc/src/main/java/org/apache/eagle/gc/model/GCPausedEvent.java
new file mode 100644
index 0000000..8ecdd02
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/model/GCPausedEvent.java
@@ -0,0 +1,249 @@
+/*
+ *
+ *  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.eagle.gc.model;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.SortedMap;
+import java.util.TreeMap;
+
+/**
+ * timestamp 		    	long
+ * eventType        		string
+ * pausedGCTimeSec 	    	double
+ * youngAreaGCed	     	boolean
+ * youngUsedHeapK   		long
+ * youngTotalHeapK   		long
+ * tenuredAreaGCed  		boolean
+ * tenuredUsedHeapK  		long
+ * tenuredTotalHeapK 		long
+ * permAreaGCed	      		boolean
+ * permUsedHeapK	    	long
+ * permTotalHeapK   		long
+ * totalHeapUsageAvailable  boolean
+ * usedTotalHeapK	   		long
+ * totalHeapK		   		long
+ * logLine					string
+ */
+
+public class GCPausedEvent {
+
+	private long timestamp;
+	private String eventType;
+	private double pausedGCTimeSec;
+
+	private boolean youngAreaGCed;
+	private long youngUsedHeapK;
+	private long youngTotalHeapK;
+
+	private boolean tenuredAreaGCed;
+	private long tenuredUsedHeapK;
+	private long tenuredTotalHeapK;
+
+	private boolean permAreaGCed;
+	private long permUsedHeapK;
+	private long permTotalHeapK;
+
+	private boolean totalHeapUsageAvailable;
+	private long usedTotalHeapK;
+	private long totalHeapK;
+	private String logLine;
+
+	public long getTimestamp() {
+		return timestamp;
+	}
+
+	public void setTimestamp(long timestamp) {
+		this.timestamp = timestamp;
+	}
+
+	public String getEventType() {
+		return eventType;
+	}
+
+	public void setEventType(String eventType) {
+		this.eventType = eventType;
+	}
+
+	public double getPausedGCTimeSec() {
+		return pausedGCTimeSec;
+	}
+
+	public void setPausedGCTimeSec(double pausedGCTimeSec) {
+		this.pausedGCTimeSec = pausedGCTimeSec;
+	}
+
+	public boolean isYoungAreaGCed() {
+		return youngAreaGCed;
+	}
+
+	public void setYoungAreaGCed(boolean youngAreaGCed) {
+		this.youngAreaGCed = youngAreaGCed;
+	}
+
+	public long getYoungUsedHeapK() {
+		return youngUsedHeapK;
+	}
+
+	public void setYoungUsedHeapK(long youngUsedHeapK) {
+		this.youngUsedHeapK = youngUsedHeapK;
+	}
+
+	public long getYoungTotalHeapK() {
+		return youngTotalHeapK;
+	}
+
+	public void setYoungTotalHeapK(long youngTotalHeapK) {
+		this.youngTotalHeapK = youngTotalHeapK;
+	}
+
+	public boolean isTenuredAreaGCed() {
+		return tenuredAreaGCed;
+	}
+
+	public void setTenuredAreaGCed(boolean tenuredAreaGCed) {
+		this.tenuredAreaGCed = tenuredAreaGCed;
+	}
+
+	public long getTenuredUsedHeapK() {
+		return tenuredUsedHeapK;
+	}
+
+	public void setTenuredUsedHeapK(long tenuredUsedHeapK) {
+		this.tenuredUsedHeapK = tenuredUsedHeapK;
+	}
+
+	public long getTenuredTotalHeapK() {
+		return tenuredTotalHeapK;
+	}
+
+	public void setTenuredTotalHeapK(long tenuredTotalHeapK) {
+		this.tenuredTotalHeapK = tenuredTotalHeapK;
+	}
+
+	public boolean isPermAreaGCed() {
+		return permAreaGCed;
+	}
+
+	public void setPermAreaGCed(boolean permAreaGCed) {
+		this.permAreaGCed = permAreaGCed;
+	}
+
+	public long getPermUsedHeapK() {
+		return permUsedHeapK;
+	}
+
+	public void setPermUsedHeapK(long permUsedHeapK) {
+		this.permUsedHeapK = permUsedHeapK;
+	}
+
+	public long getPermTotalHeapK() {
+		return permTotalHeapK;
+	}
+
+	public void setPermTotalHeapK(long permTotalHeapK) {
+		this.permTotalHeapK = permTotalHeapK;
+	}
+
+	public String getLogLine() {
+		return logLine;
+	}
+
+	public void setLogLine(String logLine) {
+		this.logLine = logLine;
+	}
+
+	public boolean isTotalHeapUsageAvailable() {
+		return totalHeapUsageAvailable;
+	}
+
+	public void setTotalHeapUsageAvailable(boolean totalHeapUsageAvailable) {
+		this.totalHeapUsageAvailable = totalHeapUsageAvailable;
+	}
+
+	public long getUsedTotalHeapK() {
+		return usedTotalHeapK;
+	}
+
+	public void setUsedTotalHeapK(long usedTotalHeapK) {
+		this.usedTotalHeapK = usedTotalHeapK;
+	}
+
+	public long getTotalHeapK() {
+		return totalHeapK;
+	}
+
+	public void setTotalHeapK(long totalHeapK) {
+		this.totalHeapK = totalHeapK;
+	}
+
+	public GCPausedEvent() {
+
+	}
+
+	public GCPausedEvent(Map<String, Object> map) {
+		this.timestamp = (Long)map.get("timestamp");
+		this.eventType = (String)map.get("eventType");
+		this.pausedGCTimeSec = (Double)map.get("pausedGCTimeSec");
+
+		this.youngAreaGCed = (Boolean)map.get("youngAreaGCed");
+		this.youngUsedHeapK = (Long)map.get("youngUsedHeapK");
+		this.youngTotalHeapK = (Long)map.get("youngTotalHeapK");
+
+		this.tenuredAreaGCed = (Boolean)map.get("tenuredAreaGCed");
+		this.tenuredUsedHeapK = (Long)map.get("tenuredUsedHeapK");
+		this.tenuredTotalHeapK = (Long)map.get("tenuredTotalHeapK");
+
+		this.permAreaGCed = (Boolean)map.get("permAreaGCed");
+		this.permUsedHeapK = (Long)map.get("permUsedHeapK");
+		this.permTotalHeapK = (Long)map.get("permTotalHeapK");
+
+		this.totalHeapUsageAvailable = (Boolean)map.get("totalHeapUsageAvailable");
+		this.usedTotalHeapK = (Long)map.get("usedTotalHeapK");
+		this.totalHeapK = (Long)map.get("totalHeapK");
+
+		this.logLine = (String)map.get("logLine");
+	}
+
+	public SortedMap<String, Object> toMap() {
+		SortedMap<String, Object> map = new TreeMap<>();
+		map.put("timestamp", timestamp);
+		map.put("eventType", eventType);
+		map.put("pausedGCTimeSec", pausedGCTimeSec);
+
+		map.put("youngAreaGCed", youngAreaGCed);
+		map.put("youngUsedHeapK", youngUsedHeapK);
+		map.put("youngTotalHeapK", youngTotalHeapK);
+
+		map.put("tenuredAreaGCed", tenuredAreaGCed);
+		map.put("tenuredUsedHeapK", tenuredUsedHeapK);
+		map.put("tenuredTotalHeapK", tenuredTotalHeapK);
+
+		map.put("permAreaGCed",  permAreaGCed);
+		map.put("permUsedHeapK", permUsedHeapK);
+		map.put("permTotalHeapK", permTotalHeapK);
+
+		map.put("totalHeapUsageAvailable", totalHeapUsageAvailable);
+		map.put("usedTotalHeapK", usedTotalHeapK);
+		map.put("totalHeapK", totalHeapK);
+
+		map.put("logLine", logLine);
+		return map;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/DateTimeParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/DateTimeParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/DateTimeParser.java
new file mode 100644
index 0000000..23201f3
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/DateTimeParser.java
@@ -0,0 +1,37 @@
+/*
+ *
+ *    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.eagle.gc.parser;
+
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+public class DateTimeParser {
+	/**
+	 * sample 2014-06-04T22:21:19.158-0700
+	 * see http://docs.oracle.com/javase/6/docs/api/java/text/SimpleDateFormat.html#year
+	 * @param dateTimeString
+	 * @return
+	 */
+	public static Date parseDateTimeString(String dateTimeString) throws Exception{
+		SimpleDateFormat sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSSZ");
+		Date d = sdf.parse(dateTimeString);
+		return d;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCEventParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCEventParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCEventParser.java
new file mode 100644
index 0000000..89bad5d
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCEventParser.java
@@ -0,0 +1,31 @@
+/*
+ *
+ *    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.eagle.gc.parser;
+
+import org.apache.eagle.gc.model.GCPausedEvent;
+
+/**
+ * Parse GC log file to object and return GCPausedEvent
+ * the implementation class should be thread safe, as this class is used assuming it's thread safe
+ * only applicable to the following GC log format which probably generated by GC options "-Xmx6G -Xms2G -XX:+PrintGCDetails -XX:+PrintGCDateStamps -Xloggc=gc.log -XX:+UseConcMarkSweepGC"
+ */
+public interface GCEventParser {
+	GCPausedEvent parse(String line) throws Exception;
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCEventParserEnum.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCEventParserEnum.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCEventParserEnum.java
new file mode 100644
index 0000000..1e18d21
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCEventParserEnum.java
@@ -0,0 +1,46 @@
+/*
+ *
+ *    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.eagle.gc.parser;
+
+import org.apache.eagle.gc.parser.tenured.*;
+import org.apache.eagle.gc.parser.full.ConcurrentModeFailureParser;
+import org.apache.eagle.gc.parser.full.NormalFullGCParser;
+import org.apache.eagle.gc.parser.full.ParaNewPromotionFailureParser;
+import org.apache.eagle.gc.parser.young.ParaNewParser;
+
+public enum GCEventParserEnum {
+	paraNew(new ParaNewParser()),
+	cmsInitialMark(new CMSInitialMarkParser()),
+	cmsRemark(new CMSRemarkParser()),
+	fullGC(new NormalFullGCParser()),
+	paraNewPromotionFailed(new ParaNewPromotionFailureParser()),
+	concurrentModeFailed(new ConcurrentModeFailureParser()),
+	other(new OtherLogFormatParser());
+
+	private GCEventParser parser;
+	
+	GCEventParserEnum(GCEventParser parser){
+		this.parser = parser;
+	}
+
+	public GCEventParser getParser(){
+		return parser;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCType.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCType.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCType.java
new file mode 100644
index 0000000..74141b2
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/GCType.java
@@ -0,0 +1,26 @@
+/*
+ *
+ *    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.eagle.gc.parser;
+
+public enum GCType {
+    YoungGC,
+    TenuredGC,
+    FullGC,
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/OtherLogFormatParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/OtherLogFormatParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/OtherLogFormatParser.java
new file mode 100644
index 0000000..1da99bd
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/OtherLogFormatParser.java
@@ -0,0 +1,39 @@
+/*
+ *
+ *    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.eagle.gc.parser;
+
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.parser.exception.IgnoredLogFormatException;
+import org.apache.eagle.gc.parser.exception.UnrecognizedLogFormatException;
+
+public class OtherLogFormatParser implements GCEventParser {
+
+	public OtherLogFormatParser(){
+	}
+	
+	@Override
+	public GCPausedEvent parse(String line) throws Exception{
+		if(line.contains("CMS-concurrent-mark-start") || line.contains("CMS-concurrent-preclean")
+			|| line.contains("CMS-concurrent-abortable-preclean") || line.contains("CMS-concurrent-sweep") || line.contains("CMS-concurrent-reset")) {
+			throw new IgnoredLogFormatException("The log is non stop the world event, just ignore it, log: "+ line);
+		}
+		throw new UnrecognizedLogFormatException("The log pattern is unknown, log: " + line);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/exception/IgnoredLogFormatException.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/exception/IgnoredLogFormatException.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/exception/IgnoredLogFormatException.java
new file mode 100644
index 0000000..babf145
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/exception/IgnoredLogFormatException.java
@@ -0,0 +1,39 @@
+/*
+ *
+ *    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.eagle.gc.parser.exception;
+
+public class IgnoredLogFormatException extends Exception {
+
+    public IgnoredLogFormatException() {
+        super();
+    }
+
+    public IgnoredLogFormatException(String message) {
+        super(message);
+    }
+
+    public IgnoredLogFormatException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public IgnoredLogFormatException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/exception/UnrecognizedLogFormatException.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/exception/UnrecognizedLogFormatException.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/exception/UnrecognizedLogFormatException.java
new file mode 100644
index 0000000..deb5134
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/exception/UnrecognizedLogFormatException.java
@@ -0,0 +1,39 @@
+/*
+ *
+ *    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.eagle.gc.parser.exception;
+
+public class UnrecognizedLogFormatException extends Exception {
+
+    public UnrecognizedLogFormatException() {
+        super();
+    }
+
+    public UnrecognizedLogFormatException(String message) {
+        super(message);
+    }
+
+    public UnrecognizedLogFormatException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public UnrecognizedLogFormatException(Throwable cause) {
+        super(cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/ConcurrentModeFailureParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/ConcurrentModeFailureParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/ConcurrentModeFailureParser.java
new file mode 100644
index 0000000..a2541d8
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/ConcurrentModeFailureParser.java
@@ -0,0 +1,62 @@
+/*
+ *
+ *    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.eagle.gc.parser.full;
+
+import org.apache.eagle.gc.parser.GCType;
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.parser.GCEventParser;
+
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * "Concurrent mode failure can be a separate line after ParaNew Promotion Failure
+ (concurrent mode failure): 89131378K->75055239K(97517568K), 430.8303930 secs] 91834503K->75055239K(100348736K), [CMS Perm : 54559K->54414K(83968K)], 431.5362150 secs] [Times: user=574.23 sys=0.00, real=431.47 secs]
+ */
+ 
+public class ConcurrentModeFailureParser implements GCEventParser {
+
+	private Pattern cmfPattern = Pattern.compile("\\(concurrent mode failure\\): ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), (?:[\\.0-9]+) secs\\] ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), \\[CMS Perm : ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\)\\], ([\\.0-9]+) secs\\] \\[Times: user=(?:[\\.0-9]+) sys=(?:[\\.0-9]+), real=([\\.0-9]+) secs\\]");
+	
+	public ConcurrentModeFailureParser(){
+	}
+	
+	@Override
+	public GCPausedEvent parse(String line) throws Exception{
+		GCPausedEvent bean = null;
+		boolean matched;
+		Matcher m = cmfPattern.matcher(line);
+		matched = m.find();
+		if(matched){
+			bean = new GCPausedEvent();
+			bean.setEventType(GCType.FullGC.name());
+			bean.setTenuredAreaGCed(true);
+			bean.setTenuredUsedHeapK(Integer.parseInt(m.group(1)));
+			bean.setTenuredTotalHeapK(Integer.parseInt(m.group(2)));
+			bean.setPermAreaGCed(true);
+			bean.setPermUsedHeapK(Integer.parseInt(m.group(7)));
+			bean.setPermTotalHeapK(Integer.parseInt(m.group(9)));
+
+			bean.setPausedGCTimeSec(Double.parseDouble(m.group(10)));
+			bean.setLogLine(line);
+		}
+		return bean;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/NormalFullGCParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/NormalFullGCParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/NormalFullGCParser.java
new file mode 100644
index 0000000..a8a0eec
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/NormalFullGCParser.java
@@ -0,0 +1,79 @@
+/*
+ *
+ *    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.eagle.gc.parser.full;
+
+import org.apache.eagle.gc.parser.GCType;
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.parser.DateTimeParser;
+import org.apache.eagle.gc.parser.GCEventParser;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+
+public class NormalFullGCParser implements GCEventParser {
+	List<Pattern> patterns = new ArrayList<Pattern>(); 
+	
+	/***
+	 * 2014-08-13T12:22:25.488-0700: 144.526: [Full GC2014-08-13T12:22:25.488-0700: 144.526: [CMS: 9845647K->10115891K(97517568K), 14.2064400 secs] 10215536K->10115891K(100348736K), [CMS Perm : 24119K->24107K(24320K)], 14.2066090 secs] [Times: user=13.86 sys=0.32, real=14.20 secs]
+	 */	
+	public NormalFullGCParser() {
+		patterns.add(Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[Full GC(?:[^\\s]+): (?:[\\.0-9]+): \\[CMS: ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), (?:[\\.0-9]+) secs\\] ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), \\[CMS Perm : ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\)\\], ([\\.0-9]+) secs\\] \\[Times: user=(?:[\\.0-9]+) sys=(?:[\\.0-9]+), real=([\\.0-9]+) secs\\]"));
+	}
+	
+	@Override
+	public GCPausedEvent parse(String line) throws Exception{
+		GCPausedEvent bean = null;
+		boolean matched = false;
+		Matcher m = null;		
+		for (Pattern pattern : patterns) {
+			m = pattern.matcher(line);
+			matched = m.find();
+			if (matched) break;
+		}
+		
+		if(matched){
+			bean = new GCPausedEvent();
+			// date time portion
+			String dateTimeString = m.group(1);
+			Date d = DateTimeParser.parseDateTimeString(dateTimeString);
+			bean.setTimestamp(d.getTime());
+			bean.setEventType(GCType.FullGC.name());
+
+			bean.setTenuredAreaGCed(true);
+			bean.setTenuredUsedHeapK(Integer.parseInt(m.group(2)));
+			bean.setTenuredTotalHeapK(Integer.parseInt(m.group(4)));
+
+			bean.setPermAreaGCed(true);
+			bean.setPermUsedHeapK(Integer.parseInt(m.group(8)));
+			bean.setPermTotalHeapK(Integer.parseInt(m.group(10)));
+
+			bean.setPausedGCTimeSec(Double.parseDouble(m.group(11)));
+			bean.setLogLine(line);
+		}
+		return bean;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/ParaNewPromotionFailureParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/ParaNewPromotionFailureParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/ParaNewPromotionFailureParser.java
new file mode 100644
index 0000000..e2c82cb
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/full/ParaNewPromotionFailureParser.java
@@ -0,0 +1,106 @@
+/*
+ *
+ *    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.eagle.gc.parser.full;
+
+import org.apache.eagle.gc.parser.GCType;
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.parser.DateTimeParser;
+import org.apache.eagle.gc.parser.GCEventParser;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+   paraNew Promotion failure patterns
+   1) With Tenured And Perm GC part
+      a) 2014-07-08T21:52:32.910-0700: 2935883.704: [GC 2935883.704: [ParNew (promotion failed): 2827840K->2824012K(2831168K), 0.8317310 secs]2935884.536: [CMS: 71633438K->38250633K(97517568K), 211.7852880 secs] 74430253K->38250633K(100348736K), [CMS Perm : 54880K->54867K(83968K)], 212.6173060 secs] [Times: user=0.00 sys=214.88, real=212.59 secs]
+      b) 2014-09-25T18:25:25.424-0700: 2568209.488: [GC2014-09-25T18:25:25.424-0700: 2568209.488: [ParNew (promotion failed): 4665858K->4627344K(4718592K), 1.2918500 secs]2014-09-25T18:25:26.716-0700: 2568210.780: [CMS: 74802314K->42339736K(95420416K), 230.2937730 secs] 79433462K->42339736K(100139008K), [CMS Perm : 49896K->49865K(83316K)], 231.5926250 secs] [Times: user=0.00 sys=233.73, real=231.56 secs]
+   2) Without Tenured pattern
+ 	  c) 2014-09-16T02:10:29.456-0700: 1732113.520: [GC2014-09-16T02:10:29.456-0700: 1732113.520: [ParNew (promotion failed): 4703469K->4718592K(4718592K), 0.9636440 secs]2014-09-16T02:10:30.420-0700: 1732114.484: [CMS2014-09-16T02:10:48.794-0700: 1732132.858: [CMS-concurrent-mark: 28.139/29.793 secs] [Times: user=214.69 sys=8.41, real=29.79 secs]
+      d) 2014-06-05T22:57:29.955-0700: 88580.749: [GC 88580.749: [ParNew (promotion failed): 2809111K->2831168K(2831168K), 0.6941530 secs]88581.443: [CMS2014-06-05T22:57:57.509-0700: 88608.303: [CMS-concurrent-sweep: 48.562/51.183 secs] [Times: user=138.07 sys=15.38, real=51.18 secs]
+ */
+public class ParaNewPromotionFailureParser implements GCEventParser {
+	
+	List<Pattern> withTenuredGCPatterns = new ArrayList<Pattern>();
+	List<Pattern> withoutTenuredGCPatterns = new ArrayList<Pattern>();
+
+	public ParaNewPromotionFailureParser() {
+		withTenuredGCPatterns.add(Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[GC (?:[\\.0-9]+): \\[ParNew \\(promotion failed\\): ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), (?:[\\.0-9]+) secs\\](?:[\\.0-9]+): \\[CMS: ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), (?:[\\.0-9]+) secs\\] (?:[0-9]+)K->(?:[0-9]+)K\\((?:[0-9]+)K\\), \\[CMS Perm : ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\)\\], ([\\.0-9]+) secs\\] \\[Times: user=(?:[\\.0-9]+) sys=(?:[\\.0-9]+), real=([\\.0-9]+) secs\\]"));
+		withTenuredGCPatterns.add(Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[GC(?:[^\\s]+): (?:[\\.0-9]+): \\[ParNew \\(promotion failed\\): ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), (?:[\\.0-9]+) secs\\](?:[^\\s]+): (?:[\\.0-9]+): \\[CMS: ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), (?:[\\.0-9]+) secs\\] (?:[0-9]+)K->(?:[0-9]+)K\\((?:[0-9]+)K\\), \\[CMS Perm : ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\)\\], ([\\.0-9]+) secs\\] \\[Times: user=(?:[\\.0-9]+) sys=(?:[\\.0-9]+), real=([\\.0-9]+) secs\\]"));
+
+		withoutTenuredGCPatterns.add(Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[GC (?:[\\.0-9]+): \\[ParNew \\(promotion failed\\): ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), (?:[\\.0-9]+) secs\\](?:[\\.0-9]+): \\[CMS(?:[^\\s]+): (?:[\\.0-9]+): \\[(?:[^\\s]+): (?:[\\.0-9]+)/(?:[\\.0-9]+) secs\\] \\[Times: user=(?:[\\.0-9]+) sys=(?:[\\.0-9]+), real=([\\.0-9]+) secs\\]"));
+		withoutTenuredGCPatterns.add(Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[GC(?:[^\\s]+): (?:[\\.0-9]+): \\[ParNew \\(promotion failed\\): ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), (?:[\\.0-9]+) secs\\](?:[^\\s]+): (?:[\\.0-9]+): \\[(?:CMS[^\\s]+): (?:[\\.0-9]+): \\[(?:[^\\s]+): (?:[\\.0-9]+)/(?:[\\.0-9]+) secs\\] \\[Times: user=(?:[\\.0-9]+) sys=(?:[\\.0-9]+), real=([\\.0-9]+) secs\\]"));
+	}
+	
+	@Override
+	public GCPausedEvent parse(String line) throws Exception{
+		GCPausedEvent bean = null;
+		boolean matched = false;
+		Matcher m = null;
+		for (Pattern pattern : withTenuredGCPatterns) {
+			m = pattern.matcher(line);
+			matched = m.find();
+			if (matched) break;
+		}
+		if(matched){
+			bean = new GCPausedEvent();
+			String dateTimeString = m.group(1);
+			Date d = DateTimeParser.parseDateTimeString(dateTimeString);
+			bean.setTimestamp(d.getTime());
+			bean.setEventType(GCType.FullGC.name());
+
+			bean.setYoungAreaGCed(true);
+			bean.setYoungUsedHeapK(Integer.parseInt(m.group(2)));
+			bean.setYoungTotalHeapK(Integer.parseInt(m.group(4)));
+			bean.setTenuredAreaGCed(true);
+			bean.setTenuredUsedHeapK(Integer.parseInt(m.group(5)));
+			bean.setTenuredTotalHeapK(Integer.parseInt(m.group(7)));
+			bean.setPermAreaGCed(true);
+			bean.setPermUsedHeapK(Integer.parseInt(m.group(8)));
+			bean.setPermTotalHeapK(Integer.parseInt(m.group(10)));
+			bean.setPausedGCTimeSec(Double.parseDouble(m.group(11)));
+
+			bean.setLogLine(line);
+			return bean;
+		}
+
+		for (Pattern pattern : withoutTenuredGCPatterns) {
+			m = pattern.matcher(line);
+			matched = m.find();
+			if (matched) break;
+		}
+		if(matched){
+			bean = new GCPausedEvent();
+			String dateTimeString = m.group(1);
+			Date d = DateTimeParser.parseDateTimeString(dateTimeString);
+			bean.setTimestamp(d.getTime());
+			bean.setYoungAreaGCed(true);
+			bean.setYoungUsedHeapK(Integer.parseInt(m.group(2)));
+			bean.setYoungTotalHeapK(Integer.parseInt(m.group(4)));
+			bean.setPausedGCTimeSec(Double.parseDouble(m.group(5)));
+			bean.setLogLine(line);
+			return bean;
+		}
+		return bean;
+	}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/tenured/CMSInitialMarkParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/tenured/CMSInitialMarkParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/tenured/CMSInitialMarkParser.java
new file mode 100644
index 0000000..ac1d0dd
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/tenured/CMSInitialMarkParser.java
@@ -0,0 +1,63 @@
+/*
+ *
+ *    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.eagle.gc.parser.tenured;
+
+import org.apache.eagle.gc.parser.GCType;
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.parser.DateTimeParser;
+import org.apache.eagle.gc.parser.GCEventParser;
+
+import java.util.Date;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class CMSInitialMarkParser implements GCEventParser {
+	/***
+	 * 2014-06-04T22:47:31.218-0700: 1582.012: [GC [1 CMS-initial-mark: 78942227K(97517568K)] 79264643K(100348736K), 0.2334170 secs] [Times: user=0.23 sys=0.00, real=0.24 secs]
+	 */
+	Pattern cmsInitialMarkPattern = Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[GC \\[1 CMS-initial-mark: ([0-9]+)K\\(([0-9]+)K\\)\\] (?:[0-9]+)K\\((?:[0-9]+)K\\), ([\\.0-9]+) secs\\] \\[Times: user=([\\.0-9]+) sys=([\\.0-9]+), real=([\\.0-9]+) secs\\]");
+
+	public CMSInitialMarkParser(){
+	}
+	
+	@Override
+	public GCPausedEvent parse(String line) throws Exception{
+		GCPausedEvent bean = null;
+		boolean matched = false;
+		Matcher m = cmsInitialMarkPattern.matcher(line);
+		matched = m.find();
+		if(matched){
+			bean = new GCPausedEvent();
+			// date time portion
+			String dateTimeString = m.group(1);
+			Date d = DateTimeParser.parseDateTimeString(dateTimeString);
+			bean.setTimestamp(d.getTime());
+			bean.setEventType(GCType.TenuredGC.name());
+
+			// tenured memory
+			bean.setTenuredUsedHeapK(Integer.parseInt(m.group(2)));
+			bean.setTenuredTotalHeapK(Integer.parseInt(m.group(3)));
+			bean.setPausedGCTimeSec(Double.parseDouble(m.group(4)));
+			
+			bean.setLogLine(line);
+		}
+		return bean;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/tenured/CMSRemarkParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/tenured/CMSRemarkParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/tenured/CMSRemarkParser.java
new file mode 100644
index 0000000..b9b70e2
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/tenured/CMSRemarkParser.java
@@ -0,0 +1,72 @@
+/*
+ *
+ *    Licensed to the Apache Software Foundation (ASF) under one or more
+ *    contributor license agreements.  See the NOTICE file distributed with
+ *    this work for additional information regarding copyright ownership.
+ *    The ASF licenses this file to You under the Apache License, Version 2.0
+ *    (the "License"); you may not use this file except in compliance with
+ *    the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *    Unless required by applicable law or agreed to in writing, software
+ *    distributed under the License is distributed on an "AS IS" BASIS,
+ *    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *    See the License for the specific language governing permissions and
+ *    limitations under the License.
+ *
+ */
+
+package org.apache.eagle.gc.parser.tenured;
+
+import org.apache.eagle.gc.parser.GCType;
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.parser.DateTimeParser;
+import org.apache.eagle.gc.parser.GCEventParser;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class CMSRemarkParser implements GCEventParser {
+
+	/**
+	 * 2014-06-04T22:49:50.603-0700: 1721.397: [GC[YG occupancy: 2777944 K (2831168 K)]1721.398: [Rescan (parallel) , 0.1706730 secs]1721.568: [weak refs processing, 0.0156130 secs] [1 CMS-remark: 83730081K(97517568K)] 86508026K(100348736K), 0.1868130 secs] [Times: user=3.04 sys=0.01, real=0.18 secs]
+	 */
+
+	List<Pattern> cmsRemarkPatterns = new ArrayList<Pattern>();
+	
+	public CMSRemarkParser() {
+		cmsRemarkPatterns.add(Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[GC.*\\[1 CMS-remark: ([0-9]+)K\\(([0-9]+)K\\)\\] (?:[0-9]+)K\\((?:[0-9]+)K\\), ([\\.0-9]+) secs\\] \\[Times: user=([\\.0-9]+) sys=([\\.0-9]+), real=([\\.0-9]+) secs\\]"));
+	}
+
+	@Override
+	public GCPausedEvent parse(String line) throws Exception{
+		GCPausedEvent bean = null;
+		boolean matched = false;
+		Matcher m = null;
+		for (Pattern pattern : cmsRemarkPatterns) {
+			m = pattern.matcher(line);
+			matched = m.find();
+			if (matched) break;
+		}
+		if(matched){
+			bean = new GCPausedEvent();
+			// date time portion
+			String dateTimeString = m.group(1);
+			Date d = DateTimeParser.parseDateTimeString(dateTimeString);
+			bean.setTimestamp(d.getTime());
+			bean.setEventType(GCType.TenuredGC.name());
+
+			bean.setTenuredAreaGCed(true);
+			bean.setTenuredUsedHeapK(Integer.parseInt(m.group(2)));
+			bean.setTenuredTotalHeapK(Integer.parseInt(m.group(3)));
+
+			bean.setPausedGCTimeSec(Double.parseDouble(m.group(4)));
+			bean.setLogLine(line);
+		}
+		return bean;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/parser/young/ParaNewParser.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/parser/young/ParaNewParser.java b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/young/ParaNewParser.java
new file mode 100644
index 0000000..ed2dfbf
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/parser/young/ParaNewParser.java
@@ -0,0 +1,77 @@
+/*
+ *
+ *    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.eagle.gc.parser.young;
+
+import org.apache.eagle.gc.parser.GCType;
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.parser.DateTimeParser;
+import org.apache.eagle.gc.parser.GCEventParser;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class ParaNewParser implements GCEventParser {
+	/**
+	   a) 2014-06-04T22:21:19.158-0700: 9.952: [GC 9.952: [ParNew: 2365777K->5223K(2831168K), 0.0155080 secs] 2365777K->5223K(100348736K), 0.0156030 secs] [Times: user=0.08 sys=0.05, real=0.02 secs]
+	   b) 2014-08-27T01:17:36.211-0700: 940.275: [GC2014-08-27T01:17:36.212-0700: 940.275: [ParNew: 4718592K->524288K(4718592K), 1.1674000 secs] 53266668K->49584240K(100139008K), 1.1676470 secs] [Times: user=19.35 sys=0.26, real=1.16 secs]
+	**/
+		
+	List<Pattern> patterns = new ArrayList<Pattern>(); 
+	
+	public ParaNewParser() {
+		patterns.add(Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[GC (?:[\\.0-9]+): \\[ParNew: ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), ([\\.0-9]+) secs\\] ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), ([\\.0-9]+) secs\\] \\[Times: user=([\\.0-9]+) sys=([\\.0-9]+), real=([\\.0-9]+) secs\\]"));
+		patterns.add(Pattern.compile("^([^\\s]+): (?:[\\.0-9]+): \\[GC(?:[^\\s]+): (?:[\\.0-9]+): \\[ParNew: ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), ([\\.0-9]+) secs\\] ([0-9]+)K->([0-9]+)K\\(([0-9]+)K\\), ([\\.0-9]+) secs\\] \\[Times: user=([\\.0-9]+) sys=([\\.0-9]+), real=([\\.0-9]+) secs\\]"));
+	}
+	
+	@Override
+	public GCPausedEvent parse(String line) throws Exception{
+		GCPausedEvent bean = null;
+		boolean matched = false;
+		Matcher m = null;		
+		for (Pattern pattern : patterns) {
+			m = pattern.matcher(line);
+			matched = m.find();
+			if (matched) break;
+		}
+		
+		if(matched){
+			bean = new GCPausedEvent();
+			// date time portion
+			String dateTimeString = m.group(1);
+			Date d = DateTimeParser.parseDateTimeString(dateTimeString);
+			bean.setTimestamp(d.getTime());
+			bean.setEventType(GCType.YoungGC.name());
+
+			bean.setYoungAreaGCed(true);
+			bean.setYoungUsedHeapK(Integer.parseInt(m.group(2)));
+			bean.setYoungTotalHeapK(Integer.parseInt(m.group(4)));
+			bean.setTotalHeapUsageAvailable(true);
+			bean.setUsedTotalHeapK(Integer.parseInt(m.group(6)));
+			bean.setTotalHeapK(Integer.parseInt(m.group(8)));
+
+			bean.setPausedGCTimeSec(Double.parseDouble(m.group(9)));
+			bean.setLogLine(line);
+		}
+		return bean;
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/spout/GCLogDeserializer.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/spout/GCLogDeserializer.java b/eagle-gc/src/main/java/org/apache/eagle/gc/spout/GCLogDeserializer.java
new file mode 100644
index 0000000..f1b3105
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/spout/GCLogDeserializer.java
@@ -0,0 +1,39 @@
+/*
+ *
+ *    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.eagle.gc.spout;
+
+import org.apache.eagle.dataproc.impl.storm.kafka.SpoutKafkaMessageDeserializer;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.util.Properties;
+
+public class GCLogDeserializer implements SpoutKafkaMessageDeserializer {
+
+    private Properties props;
+
+    public  GCLogDeserializer(Properties props){
+        this.props = props;
+    }
+
+    @Override
+    public Object deserialize(byte[] arg0) {
+        return Bytes.toString(arg0);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/java/org/apache/eagle/gc/stream/GCStreamBuilder.java
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/java/org/apache/eagle/gc/stream/GCStreamBuilder.java b/eagle-gc/src/main/java/org/apache/eagle/gc/stream/GCStreamBuilder.java
new file mode 100644
index 0000000..957a578
--- /dev/null
+++ b/eagle-gc/src/main/java/org/apache/eagle/gc/stream/GCStreamBuilder.java
@@ -0,0 +1,46 @@
+/*
+ *
+ *    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.eagle.gc.stream;
+
+import org.apache.eagle.gc.model.GCPausedEvent;
+import org.apache.eagle.gc.parser.GCEventParserEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class GCStreamBuilder {
+	private static Logger LOGGER = LoggerFactory.getLogger(GCStreamBuilder.class);
+
+	public GCPausedEvent build(String line) throws Exception{
+		GCPausedEvent event;
+		event = GCEventParserEnum.paraNew.getParser().parse(line);
+		if (event != null) return event;
+		event = GCEventParserEnum.cmsInitialMark.getParser().parse(line);
+		if (event != null) return event;
+		event = GCEventParserEnum.cmsRemark.getParser().parse(line);
+		if (event != null) return event;
+		event = GCEventParserEnum.fullGC.getParser().parse(line);
+		if (event != null) return event;
+		event = GCEventParserEnum.concurrentModeFailed.getParser().parse(line);
+		if (event != null) return event;
+		event = GCEventParserEnum.paraNewPromotionFailed.getParser().parse(line);
+		if (event != null) return event;
+		return GCEventParserEnum.other.getParser().parse(line);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/resources/alert-gc-policy.sh
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/resources/alert-gc-policy.sh b/eagle-gc/src/main/resources/alert-gc-policy.sh
new file mode 100644
index 0000000..ffa6549
--- /dev/null
+++ b/eagle-gc/src/main/resources/alert-gc-policy.sh
@@ -0,0 +1,21 @@
+#  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.
+
+#!/bin/sh
+#### AlertDefinitionService: alert definition for NNGCLog Pause Time Long
+curl -X POST -H 'Content-Type:application/json' -H "Authorization:Basic YWRtaW46c2VjcmV0" "http://localhost:38080/eagle-service/rest/entities?serviceName=AlertDefinitionService" -d '[{"tags":{"site":"sandbox"","dataSource":"NNGCLog","policyId":"NNGCPauseTimeLong","alertExecutorId":"NNGCAlert","policyType":"siddhiCEPEngine"},"desc":"alert when namenode pause time long","policyDef":"{\"type\":\"siddhiCEPEngine\",\"expression\":\"from NNGCLogStream#window.externalTime(timestamp,10 min) select sum(pausedGCTimeSec) as sumPausedSec having sumPausedSec >= 30 insert into outputStream;\"}","dedupeDef":"","notificationDef":"","remediationDef":"","enabled":true}]'
+
+#### AlertDefinitionService: alert definition for NNGCLog Full GC
+curl -X POST -H 'Content-Type:application/json' -H "Authorization:Basic YWRtaW46c2VjcmV0" "http://localhost:38080/eagle-service/rest/entities?serviceName=AlertDefinitionService" -d '[{"tags":{"site":"sandbox","dataSource":"NNGCLog","policyId":"NNGCPauseTimeLong","alertExecutorId":"NNGCAlert","policyType":"siddhiCEPEngine"},"desc":"alert when namenode has full gc","policyDef":"{\"type\":\"siddhiCEPEngine\",\"expression\":\"from NNGCLogStream[(permAreaGCed == true)] select * insert into outputStream;\"}","dedupeDef":"","notificationDef":"","remediationDef":"","enabled":true}]'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/resources/alert-metadata-create-gc.sh
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/resources/alert-metadata-create-gc.sh b/eagle-gc/src/main/resources/alert-metadata-create-gc.sh
new file mode 100644
index 0000000..1e57c28
--- /dev/null
+++ b/eagle-gc/src/main/resources/alert-metadata-create-gc.sh
@@ -0,0 +1,27 @@
+#  Licensed to the Apache Software Foundation (ASF) under one or more
+#  contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You under the Apache License, Version 2.0
+#  (the "License"); you may not use this file except in compliance with
+#  the License.  You may obtain a copy of the License at
+#
+#       http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+#!/bin/sh
+#### AlertDataSourceService: alert streams generated from data source
+curl -X POST -H 'Content-Type:application/json' -H 'Authorization: Basic YWRtaW46c2VjcmV0' "http://localhost:38080/eagle-service/rest/entities?serviceName=AlertDataSourceService" -d '[{"prefix": "alertDataSource", "tags": {"site": "sandbox","dataSource": "NNGCLog"}}]'
+
+#### AlertStreamService: alert streams generated from data source
+curl -X POST -H 'Content-Type:application/json' -H 'Authorization: Basic YWRtaW46c2VjcmV0' "http://localhost:38080/eagle-service/rest/entities?serviceName=AlertStreamService" -d '[{"prefix":"alertStream","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream"},"desc":"alert event stream from namenode gc log"}]'
+
+#### AlertExecutorService: what alert streams are consumed by alert executor
+curl -X POST -H 'Content-Type:application/json' -H 'Authorization: Basic YWRtaW46c2VjcmV0' "http://localhost:38080/eagle-service/rest/entities?serviceName=AlertExecutorService" -d '[{"prefix":"alertExecutor","tags":{"dataSource":"NNGCLog","alertExecutorId":"NNGCAlert","streamName":"NNGCLogStream"},"desc":"alert executor for namenode gc log"}]'
+
+#### AlertStreamSchemaService: schema for event from alert stream
+curl -X POST -H 'Content-Type:application/json' -H 'Authorization: Basic YWRtaW46c2VjcmV0' "http://localhost:38080/eagle-service/rest/entities?serviceName=AlertStreamSchemaService" -d '[{"prefix":"alertStreamSchema","category":"","attrType":"long","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"timestamp"}},{"prefix":"alertStreamSchema","category":"","attrType":"string","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"eventType"}},{"prefix":"alertStreamSchema","category":"","attrType":"double","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"pausedGCTimeSec"}},{"prefix":"alertStreamSchema","category":"","attrType":"bool","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"youngAreaGCed"}},{"prefix":"alertStreamSchema","category":"","attrType":"long","attrValueResolver":"","tags":{"dataSource":"NNGCLog","
 streamName":"NNGCLogStream","attrName":"youngUsedHeapK"}},{"prefix":"alertStreamSchema","category":"","attrType":"long","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"youngTotalHeapK"}},{"prefix":"alertStreamSchema","category":"","attrType":"bool","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"tenuredAreaGCed"}},{"prefix":"alertStreamSchema","category":"","attrType":"long","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"tenuredAreaGCed"}},{"prefix":"alertStreamSchema","category":"","attrType":"long","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"tenuredTotalHeapK"}},{"prefix":"alertStreamSchema","category":"","attrType":"bool","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"permAreaGCed"}},{"prefix":"alertStreamSchema","category":"","attrType":"long
 ","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"permUsedHeapK"}},{"prefix":"alertStreamSchema","category":"","attrType":"long","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"permTotalHeapK"}},{"prefix":"alertStreamSchema","category":"","attrType":"bool","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"totalHeapUsageAvailable"}},{"prefix":"alertStreamSchema","category":"","attrType":"long","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"usedTotalHeapK"}},{"prefix":"alertStreamSchema","category":"","attrType":"bool","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"tenuredUsedHeapK"}},{"prefix":"alertStreamSchema","category":"","attrType":"long","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"totalHeapK"}},{
 "prefix":"alertStreamSchema","category":"","attrType":"string","attrValueResolver":"","tags":{"dataSource":"NNGCLog","streamName":"NNGCLogStream","attrName":"logLine"}}]'
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-eagle/blob/e0875013/eagle-gc/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/eagle-gc/src/main/resources/application.conf b/eagle-gc/src/main/resources/application.conf
new file mode 100644
index 0000000..b86fcfa
--- /dev/null
+++ b/eagle-gc/src/main/resources/application.conf
@@ -0,0 +1,60 @@
+# 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.
+{
+  "envContextConfig" : {
+    "env" : "storm",
+    "mode" : "local",
+    "topologyName" : "GCAnalysorTopology",
+    "stormConfigFile" : "gc-storm.yaml",
+    "parallelismConfig" : {
+      "kafkaMsgConsumer" : 1
+    }
+  },
+  "dataSourceConfig": {
+    "site" : "sandbox",
+    "topic" : "sandbox-namenode-gc_log",
+    "consumerGroupId" : "gc.log.eagle.consumer",
+    "zkSessionTimeoutMs" : 15000,
+    "zkRetryTimes" : 3,
+    "zkRetryInterval" : 2000,
+    "zkConnectionTimeoutMS" : 15000,
+    "fetchSize" : 1048586,
+    "deserializerClass" : "org.apache.eagle.gc.spout.GCLogDeserializer",
+    "zkConnection" : "localhost:2181",
+    "transactionZKServers" : "localhost",
+    "transactionZKPort" : "2181",
+    "transactionZKRoot" : "/consumers",
+    "transactionStateUpdateMS" : 2000,
+    "kafkaEndPoints" : "localhost:6667"
+  },
+  "eagleProps" : {
+    "site" : "sandbox",
+    "dataSource": "NNGCLog",
+    "mailHost" : "atom.corp.ebay.com",
+    "mailSmtpPort":"25",
+    "mailDebug" : "true",
+    "eagleService": {
+      "host": "localhost",
+      "port": 38080,
+      "username": "admin",
+      "password": "secret"
+    }
+  },
+  "dynamicConfigSource" : {
+    "enabled" : true,
+    "initDelayMillis" : 0,
+    "delayMillis" : 30000
+  }
+}
\ No newline at end of file