APACHE-KYLIN-2723: retire query & job metrics wrapper

Project: http://git-wip-us.apache.org/repos/asf/kylin/repo
Commit: http://git-wip-us.apache.org/repos/asf/kylin/commit/02b67ceb
Tree: http://git-wip-us.apache.org/repos/asf/kylin/tree/02b67ceb
Diff: http://git-wip-us.apache.org/repos/asf/kylin/diff/02b67ceb

Branch: refs/heads/yaho-cube-planner
Commit: 02b67cebfc294b8ba04cf66e1715326c36346f5f
Parents: 83d74c2
Author: Zhong <nju_y...@apache.org>
Authored: Thu Aug 10 19:33:07 2017 +0800
Committer: Zhong <nju_y...@apache.org>
Committed: Thu Aug 10 19:33:07 2017 +0800

----------------------------------------------------------------------
 core-job/pom.xml                                |   4 +
 .../kylin/job/metrics/JobMetricsFacade.java     |  60 +++++++++
 .../job/ExceptionRecordEventWrapper.java        |  40 ------
 .../kylin/metrics/job/JobPropertyEnum.java      |  56 ---------
 .../metrics/job/JobRecordEventWrapper.java      |  68 ----------
 .../metrics/lib/impl/RecordEventWrapper.java    |  61 ---------
 .../metrics/lib/impl/TimedRecordEvent.java      |  47 +++++++
 .../kylin/metrics/property/JobPropertyEnum.java |  56 +++++++++
 .../metrics/property/QueryCubePropertyEnum.java |  54 ++++++++
 .../metrics/property/QueryPropertyEnum.java     |  52 ++++++++
 .../metrics/property/QueryRPCPropertyEnum.java  |  51 ++++++++
 .../query/CubeSegmentRecordEventWrapper.java    |  94 --------------
 .../metrics/query/QueryRecordEventWrapper.java  |  91 --------------
 .../metrics/query/RPCRecordEventWrapper.java    |  79 ------------
 .../org/apache/kylin/engine/mr/CubingJob.java   |  50 +++++---
 .../kylin/rest/metrics/QueryMetricsFacade.java  | 124 +++++++++++++++----
 16 files changed, 454 insertions(+), 533 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-job/pom.xml
----------------------------------------------------------------------
diff --git a/core-job/pom.xml b/core-job/pom.xml
index d1444b0..abf819e 100644
--- a/core-job/pom.xml
+++ b/core-job/pom.xml
@@ -37,6 +37,10 @@
             <groupId>org.apache.kylin</groupId>
             <artifactId>kylin-core-cube</artifactId>
         </dependency>
+        <dependency>
+            <groupId>org.apache.kylin</groupId>
+            <artifactId>kylin-core-metrics</artifactId>
+        </dependency>
 
         <!-- Provided -->
         <dependency>

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-job/src/main/java/org/apache/kylin/job/metrics/JobMetricsFacade.java
----------------------------------------------------------------------
diff --git 
a/core-job/src/main/java/org/apache/kylin/job/metrics/JobMetricsFacade.java 
b/core-job/src/main/java/org/apache/kylin/job/metrics/JobMetricsFacade.java
new file mode 100644
index 0000000..9484350
--- /dev/null
+++ b/core-job/src/main/java/org/apache/kylin/job/metrics/JobMetricsFacade.java
@@ -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.
+*/
+
+package org.apache.kylin.job.metrics;
+
+import org.apache.kylin.metrics.lib.impl.RecordEvent;
+import org.apache.kylin.metrics.property.JobPropertyEnum;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class JobMetricsFacade {
+    private static final Logger logger = 
LoggerFactory.getLogger(JobMetricsFacade.class);
+
+    public static void setJobWrapper(RecordEvent metricsEvent, String 
projectName, String cubeName, String jobId,
+            String jobType, String cubingType) {
+        metricsEvent.put(JobPropertyEnum.PROJECT.toString(), projectName);
+        metricsEvent.put(JobPropertyEnum.CUBE.toString(), cubeName);
+        metricsEvent.put(JobPropertyEnum.ID_CODE.toString(), jobId);
+        metricsEvent.put(JobPropertyEnum.TYPE.toString(), jobType);
+        metricsEvent.put(JobPropertyEnum.ALGORITHM.toString(), cubingType);
+    }
+
+    public static void setJobStats(RecordEvent metricsEvent, long tableSize, 
long cubeSize, long buildDuration,
+            long waitResourceTime, double perBytesTimeCost) {
+        metricsEvent.put(JobPropertyEnum.SOURCE_SIZE.toString(), tableSize);
+        metricsEvent.put(JobPropertyEnum.CUBE_SIZE.toString(), cubeSize);
+        metricsEvent.put(JobPropertyEnum.BUILD_DURATION.toString(), 
buildDuration);
+        metricsEvent.put(JobPropertyEnum.WAIT_RESOURCE_TIME.toString(), 
waitResourceTime);
+        metricsEvent.put(JobPropertyEnum.PER_BYTES_TIME_COST.toString(), 
perBytesTimeCost);
+    }
+
+    public static void setJobStepStats(RecordEvent metricsEvent, long 
dColumnDistinct, long dDictBuilding,
+            long dCubingInmem, long dHfileConvert) {
+        
metricsEvent.put(JobPropertyEnum.STEP_DURATION_DISTINCT_COLUMNS.toString(), 
dColumnDistinct);
+        metricsEvent.put(JobPropertyEnum.STEP_DURATION_DICTIONARY.toString(), 
dDictBuilding);
+        
metricsEvent.put(JobPropertyEnum.STEP_DURATION_INMEM_CUBING.toString(), 
dCubingInmem);
+        
metricsEvent.put(JobPropertyEnum.STEP_DURATION_HFILE_CONVERT.toString(), 
dHfileConvert);
+    }
+
+    public static <T extends Throwable> void 
setJobExceptionWrapper(RecordEvent metricsEvent, String projectName,
+            String cubeName, String jobId, String jobType, String cubingType, 
Class<T> throwableClass) {
+        setJobWrapper(metricsEvent, projectName, cubeName, jobId, jobType, 
cubingType);
+        metricsEvent.put(JobPropertyEnum.EXCEPTION.toString(), 
throwableClass.getName());
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/job/ExceptionRecordEventWrapper.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/job/ExceptionRecordEventWrapper.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/job/ExceptionRecordEventWrapper.java
deleted file mode 100644
index 8d56025..0000000
--- 
a/core-metrics/src/main/java/org/apache/kylin/metrics/job/ExceptionRecordEventWrapper.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.kylin.metrics.job;
-
-import org.apache.kylin.metrics.lib.impl.RecordEvent;
-import org.apache.kylin.metrics.lib.impl.RecordEventWrapper;
-
-public class ExceptionRecordEventWrapper extends RecordEventWrapper {
-
-    public ExceptionRecordEventWrapper(RecordEvent metricsEvent) {
-        super(metricsEvent);
-    }
-
-    public <T extends Throwable> void setWrapper(String projectName, String 
cubeName, String jobId, String jobType,
-            String cubingType, Class<T> exceptionClassName) {
-        this.metricsEvent.put(JobPropertyEnum.PROJECT.toString(), projectName);
-        this.metricsEvent.put(JobPropertyEnum.CUBE.toString(), cubeName);
-        this.metricsEvent.put(JobPropertyEnum.ID_CODE.toString(), jobId);
-        this.metricsEvent.put(JobPropertyEnum.TYPE.toString(), jobType);
-        this.metricsEvent.put(JobPropertyEnum.ALGORITHM.toString(), 
cubingType);
-        this.metricsEvent.put(JobPropertyEnum.EXCEPTION.toString(), 
exceptionClassName.getName());
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/job/JobPropertyEnum.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/job/JobPropertyEnum.java 
b/core-metrics/src/main/java/org/apache/kylin/metrics/job/JobPropertyEnum.java
deleted file mode 100644
index be32424..0000000
--- 
a/core-metrics/src/main/java/org/apache/kylin/metrics/job/JobPropertyEnum.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * 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.kylin.metrics.job;
-
-import com.google.common.base.Strings;
-
-public enum JobPropertyEnum {
-    ID_CODE("JOB_ID"), PROJECT("PROJECT"), CUBE("CUBE_NAME"), 
TYPE("JOB_TYPE"), ALGORITHM("CUBING_TYPE"), STATUS(
-            "JOB_STATUS"), EXCEPTION("EXCEPTION"), //
-    SOURCE_SIZE("TABLE_SIZE"), CUBE_SIZE("CUBE_SIZE"), 
BUILD_DURATION("DURATION"), WAIT_RESOURCE_TIME(
-            "WAIT_RESOURCE_TIME"), PER_BYTES_TIME_COST("PER_BYTES_TIME_COST"), 
STEP_DURATION_DISTINCT_COLUMNS(
-                    "STEP_DURATION_DISTINCT_COLUMNS"), 
STEP_DURATION_DICTIONARY(
-                            "STEP_DURATION_DICTIONARY"), 
STEP_DURATION_INMEM_CUBING(
-                                    "STEP_DURATION_INMEM_CUBING"), 
STEP_DURATION_HFILE_CONVERT(
-                                            "STEP_DURATION_HFILE_CONVERT");
-
-    private final String propertyName;
-
-    JobPropertyEnum(String name) {
-        this.propertyName = name;
-    }
-
-    public static JobPropertyEnum getByName(String name) {
-        if (Strings.isNullOrEmpty(name)) {
-            return null;
-        }
-        for (JobPropertyEnum property : JobPropertyEnum.values()) {
-            if (property.propertyName.equals(name.toUpperCase())) {
-                return property;
-            }
-        }
-
-        return null;
-    }
-
-    @Override
-    public String toString() {
-        return propertyName;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/job/JobRecordEventWrapper.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/job/JobRecordEventWrapper.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/job/JobRecordEventWrapper.java
deleted file mode 100644
index 6cd197e..0000000
--- 
a/core-metrics/src/main/java/org/apache/kylin/metrics/job/JobRecordEventWrapper.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * 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.kylin.metrics.job;
-
-import org.apache.kylin.metrics.lib.impl.RecordEvent;
-import org.apache.kylin.metrics.lib.impl.RecordEventWrapper;
-
-public class JobRecordEventWrapper extends RecordEventWrapper {
-
-    public static final long MIN_SOURCE_SIZE = 33554432L; //32MB per block 
created by the first step
-
-    public JobRecordEventWrapper(RecordEvent metricsEvent) {
-        super(metricsEvent);
-        initStats();
-    }
-
-    public void initStats() {
-        this.metricsEvent.put(JobPropertyEnum.SOURCE_SIZE.toString(), 0L);
-        this.metricsEvent.put(JobPropertyEnum.CUBE_SIZE.toString(), 0L);
-        this.metricsEvent.put(JobPropertyEnum.BUILD_DURATION.toString(), 0L);
-        this.metricsEvent.put(JobPropertyEnum.WAIT_RESOURCE_TIME.toString(), 
0L);
-        
this.metricsEvent.put(JobPropertyEnum.STEP_DURATION_DISTINCT_COLUMNS.toString(),
 0L);
-        
this.metricsEvent.put(JobPropertyEnum.STEP_DURATION_DICTIONARY.toString(), 0L);
-        
this.metricsEvent.put(JobPropertyEnum.STEP_DURATION_INMEM_CUBING.toString(), 
0L);
-        
this.metricsEvent.put(JobPropertyEnum.STEP_DURATION_HFILE_CONVERT.toString(), 
0L);
-        this.metricsEvent.put(JobPropertyEnum.PER_BYTES_TIME_COST.toString(), 
0L);
-    }
-
-    public void setWrapper(String projectName, String cubeName, String jobId, 
String jobType, String cubingType) {
-        this.metricsEvent.put(JobPropertyEnum.PROJECT.toString(), projectName);
-        this.metricsEvent.put(JobPropertyEnum.CUBE.toString(), cubeName);
-        this.metricsEvent.put(JobPropertyEnum.ID_CODE.toString(), jobId);
-        this.metricsEvent.put(JobPropertyEnum.TYPE.toString(), jobType);
-        this.metricsEvent.put(JobPropertyEnum.ALGORITHM.toString(), 
cubingType);
-    }
-
-    public void setStats(long tableSize, long cubeSize, long buildDuration, 
long waitResourceTime,
-            double perBytesTimeCost) {
-        this.metricsEvent.put(JobPropertyEnum.SOURCE_SIZE.toString(), 
tableSize);
-        this.metricsEvent.put(JobPropertyEnum.CUBE_SIZE.toString(), cubeSize);
-        this.metricsEvent.put(JobPropertyEnum.BUILD_DURATION.toString(), 
buildDuration);
-        this.metricsEvent.put(JobPropertyEnum.WAIT_RESOURCE_TIME.toString(), 
waitResourceTime);
-        this.metricsEvent.put(JobPropertyEnum.PER_BYTES_TIME_COST.toString(), 
perBytesTimeCost);
-    }
-
-    public void setStepStats(long dColumnDistinct, long dDictBuilding, long 
dCubingInmem, long dHfileConvert) {
-        
this.metricsEvent.put(JobPropertyEnum.STEP_DURATION_DISTINCT_COLUMNS.toString(),
 dColumnDistinct);
-        
this.metricsEvent.put(JobPropertyEnum.STEP_DURATION_DICTIONARY.toString(), 
dDictBuilding);
-        
this.metricsEvent.put(JobPropertyEnum.STEP_DURATION_INMEM_CUBING.toString(), 
dCubingInmem);
-        
this.metricsEvent.put(JobPropertyEnum.STEP_DURATION_HFILE_CONVERT.toString(), 
dHfileConvert);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/lib/impl/RecordEventWrapper.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/lib/impl/RecordEventWrapper.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/lib/impl/RecordEventWrapper.java
deleted file mode 100644
index 7031129..0000000
--- 
a/core-metrics/src/main/java/org/apache/kylin/metrics/lib/impl/RecordEventWrapper.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * 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.kylin.metrics.lib.impl;
-
-import java.io.Serializable;
-
-import org.apache.kylin.metrics.lib.Record;
-
-public class RecordEventWrapper implements Serializable {
-
-    protected final RecordEvent metricsEvent;
-
-    public RecordEventWrapper(RecordEvent metricsEvent) {
-        this.metricsEvent = metricsEvent;
-
-        //Add time details
-        addTimeDetails();
-    }
-
-    private void addTimeDetails() {
-        RecordEventTimeDetail dateDetail = new 
RecordEventTimeDetail(metricsEvent.getTime());
-        metricsEvent.put(TimePropertyEnum.YEAR.toString(), 
dateDetail.year_begin_date);
-        metricsEvent.put(TimePropertyEnum.MONTH.toString(), 
dateDetail.month_begin_date);
-        metricsEvent.put(TimePropertyEnum.WEEK_BEGIN_DATE.toString(), 
dateDetail.week_begin_date);
-        metricsEvent.put(TimePropertyEnum.DAY_DATE.toString(), 
dateDetail.date);
-        metricsEvent.put(TimePropertyEnum.DAY_TIME.toString(), 
dateDetail.time);
-        metricsEvent.put(TimePropertyEnum.TIME_HOUR.toString(), 
dateDetail.hour);
-        metricsEvent.put(TimePropertyEnum.TIME_MINUTE.toString(), 
dateDetail.minute);
-        metricsEvent.put(TimePropertyEnum.TIME_SECOND.toString(), 
dateDetail.second);
-    }
-
-    public void resetTime() {
-        metricsEvent.resetTime();
-        addTimeDetails();
-    }
-
-    public Record getMetricsRecord() {
-        return metricsEvent;
-    }
-
-    @Override
-    public String toString() {
-        return metricsEvent.toString();
-    }
-}

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/lib/impl/TimedRecordEvent.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/lib/impl/TimedRecordEvent.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/lib/impl/TimedRecordEvent.java
new file mode 100644
index 0000000..a866163
--- /dev/null
+++ 
b/core-metrics/src/main/java/org/apache/kylin/metrics/lib/impl/TimedRecordEvent.java
@@ -0,0 +1,47 @@
+/*
+ * 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.kylin.metrics.lib.impl;
+
+public class TimedRecordEvent extends RecordEvent {
+
+    public TimedRecordEvent(String eventType) {
+        super(eventType);
+
+        //Add time details
+        addTimeDetails();
+    }
+
+    private void addTimeDetails() {
+        RecordEventTimeDetail dateDetail = new 
RecordEventTimeDetail(getTime());
+        put(TimePropertyEnum.YEAR.toString(), dateDetail.year_begin_date);
+        put(TimePropertyEnum.MONTH.toString(), dateDetail.month_begin_date);
+        put(TimePropertyEnum.WEEK_BEGIN_DATE.toString(), 
dateDetail.week_begin_date);
+        put(TimePropertyEnum.DAY_DATE.toString(), dateDetail.date);
+        put(TimePropertyEnum.DAY_TIME.toString(), dateDetail.time);
+        put(TimePropertyEnum.TIME_HOUR.toString(), dateDetail.hour);
+        put(TimePropertyEnum.TIME_MINUTE.toString(), dateDetail.minute);
+        put(TimePropertyEnum.TIME_SECOND.toString(), dateDetail.second);
+    }
+
+    @Override
+    public void resetTime() {
+        super.resetTime();
+        addTimeDetails();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/property/JobPropertyEnum.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/property/JobPropertyEnum.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/property/JobPropertyEnum.java
new file mode 100644
index 0000000..0453221
--- /dev/null
+++ 
b/core-metrics/src/main/java/org/apache/kylin/metrics/property/JobPropertyEnum.java
@@ -0,0 +1,56 @@
+/*
+ * 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.kylin.metrics.property;
+
+import com.google.common.base.Strings;
+
+public enum JobPropertyEnum {
+    ID_CODE("JOB_ID"), PROJECT("PROJECT"), CUBE("CUBE_NAME"), 
TYPE("JOB_TYPE"), ALGORITHM("CUBING_TYPE"), STATUS(
+            "JOB_STATUS"), EXCEPTION("EXCEPTION"), //
+    SOURCE_SIZE("TABLE_SIZE"), CUBE_SIZE("CUBE_SIZE"), 
BUILD_DURATION("DURATION"), WAIT_RESOURCE_TIME(
+            "WAIT_RESOURCE_TIME"), PER_BYTES_TIME_COST("PER_BYTES_TIME_COST"), 
STEP_DURATION_DISTINCT_COLUMNS(
+                    "STEP_DURATION_DISTINCT_COLUMNS"), 
STEP_DURATION_DICTIONARY(
+                            "STEP_DURATION_DICTIONARY"), 
STEP_DURATION_INMEM_CUBING(
+                                    "STEP_DURATION_INMEM_CUBING"), 
STEP_DURATION_HFILE_CONVERT(
+                                            "STEP_DURATION_HFILE_CONVERT");
+
+    private final String propertyName;
+
+    JobPropertyEnum(String name) {
+        this.propertyName = name;
+    }
+
+    public static JobPropertyEnum getByName(String name) {
+        if (Strings.isNullOrEmpty(name)) {
+            return null;
+        }
+        for (JobPropertyEnum property : JobPropertyEnum.values()) {
+            if (property.propertyName.equals(name.toUpperCase())) {
+                return property;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public String toString() {
+        return propertyName;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryCubePropertyEnum.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryCubePropertyEnum.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryCubePropertyEnum.java
new file mode 100644
index 0000000..d2d5bb4
--- /dev/null
+++ 
b/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryCubePropertyEnum.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+*/
+
+package org.apache.kylin.metrics.property;
+
+import com.google.common.base.Strings;
+
+public enum QueryCubePropertyEnum {
+    PROJECT("PROJECT"), CUBE("CUBE_NAME"), SEGMENT("SEGMENT_NAME"), 
CUBOID_SOURCE("CUBOID_SOURCE"), CUBOID_TARGET(
+            "CUBOID_TARGET"), IF_MATCH("IF_MATCH"), 
FILTER_MASK("FILTER_MASK"), IF_SUCCESS("IF_SUCCESS"), //
+    TIME_SUM("STORAGE_CALL_TIME_SUM"), TIME_MAX("STORAGE_CALL_TIME_MAX"), 
WEIGHT_PER_HIT("WEIGHT_PER_HIT"), CALL_COUNT(
+            "STORAGE_CALL_COUNT"), SKIP_COUNT("STORAGE_COUNT_SKIP"), 
SCAN_COUNT("STORAGE_COUNT_SCAN"), RETURN_COUNT(
+                    "STORAGE_COUNT_RETURN"), AGGR_FILTER_COUNT(
+                            "STORAGE_COUNT_AGGREGATE_FILTER"), 
AGGR_COUNT("STORAGE_COUNT_AGGREGATE");
+
+    private final String propertyName;
+
+    QueryCubePropertyEnum(String name) {
+        this.propertyName = name;
+    }
+
+    public static QueryCubePropertyEnum getByName(String name) {
+        if (Strings.isNullOrEmpty(name)) {
+            return null;
+        }
+        for (QueryCubePropertyEnum property : QueryCubePropertyEnum.values()) {
+            if (property.propertyName.equals(name.toUpperCase())) {
+                return property;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public String toString() {
+        return propertyName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryPropertyEnum.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryPropertyEnum.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryPropertyEnum.java
new file mode 100644
index 0000000..d3fd3ce
--- /dev/null
+++ 
b/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryPropertyEnum.java
@@ -0,0 +1,52 @@
+/*
+ * 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.kylin.metrics.property;
+
+import com.google.common.base.Strings;
+
+public enum QueryPropertyEnum {
+    ID_CODE("QUERY_HASH_CODE"), TYPE("QUERY_TYPE"), PROJECT("PROJECT"), 
REALIZATION("REALIZATION"), REALIZATION_TYPE(
+            "REALIZATION_TYPE"), EXCEPTION("EXCEPTION"), //
+    TIME_COST("QUERY_TIME_COST"), 
CALCITE_RETURN_COUNT("CALCITE_COUNT_RETURN"), STORAGE_RETURN_COUNT(
+            "STORAGE_COUNT_RETURN"), 
AGGR_FILTER_COUNT("CALCITE_COUNT_AGGREGATE_FILTER");
+
+    private final String propertyName;
+
+    QueryPropertyEnum(String name) {
+        this.propertyName = name;
+    }
+
+    public static QueryPropertyEnum getByName(String name) {
+        if (Strings.isNullOrEmpty(name)) {
+            return null;
+        }
+        for (QueryPropertyEnum property : QueryPropertyEnum.values()) {
+            if (property.propertyName.equals(name.toUpperCase())) {
+                return property;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public String toString() {
+        return propertyName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryRPCPropertyEnum.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryRPCPropertyEnum.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryRPCPropertyEnum.java
new file mode 100644
index 0000000..049b9ed
--- /dev/null
+++ 
b/core-metrics/src/main/java/org/apache/kylin/metrics/property/QueryRPCPropertyEnum.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.kylin.metrics.property;
+
+import com.google.common.base.Strings;
+
+public enum QueryRPCPropertyEnum {
+    PROJECT("PROJECT"), REALIZATION("REALIZATION"), RPC_SERVER("RPC_SERVER"), 
EXCEPTION("EXCEPTION"), //
+    CALL_TIME("CALL_TIME"), SKIP_COUNT("COUNT_SKIP"), 
SCAN_COUNT("COUNT_SCAN"), RETURN_COUNT(
+            "COUNT_RETURN"), AGGR_FILTER_COUNT("COUNT_AGGREGATE_FILTER"), 
AGGR_COUNT("COUNT_AGGREGATE");
+
+    private final String propertyName;
+
+    QueryRPCPropertyEnum(String name) {
+        this.propertyName = name;
+    }
+
+    public static QueryRPCPropertyEnum getByName(String name) {
+        if (Strings.isNullOrEmpty(name)) {
+            return null;
+        }
+        for (QueryRPCPropertyEnum property : QueryRPCPropertyEnum.values()) {
+            if (property.propertyName.equals(name.toUpperCase())) {
+                return property;
+            }
+        }
+
+        return null;
+    }
+
+    @Override
+    public String toString() {
+        return propertyName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/query/CubeSegmentRecordEventWrapper.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/query/CubeSegmentRecordEventWrapper.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/query/CubeSegmentRecordEventWrapper.java
deleted file mode 100644
index b5fa218..0000000
--- 
a/core-metrics/src/main/java/org/apache/kylin/metrics/query/CubeSegmentRecordEventWrapper.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * 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.kylin.metrics.query;
-
-import org.apache.kylin.metrics.lib.impl.RecordEvent;
-import org.apache.kylin.metrics.lib.impl.RecordEventWrapper;
-
-import com.google.common.base.Strings;
-
-public class CubeSegmentRecordEventWrapper extends RecordEventWrapper {
-
-    public CubeSegmentRecordEventWrapper(RecordEvent metricsEvent) {
-        super(metricsEvent);
-    }
-
-    public void setWrapper(String projectName, String cubeName, String 
segmentName, long sourceCuboidId,
-            long targetCuboidId, long filterMask) {
-        this.metricsEvent.put(PropertyEnum.PROJECT.toString(), projectName);
-        this.metricsEvent.put(PropertyEnum.CUBE.toString(), cubeName);
-        this.metricsEvent.put(PropertyEnum.SEGMENT.toString(), segmentName);
-        this.metricsEvent.put(PropertyEnum.CUBOID_SOURCE.toString(), 
sourceCuboidId);
-        this.metricsEvent.put(PropertyEnum.CUBOID_TARGET.toString(), 
targetCuboidId);
-        this.metricsEvent.put(PropertyEnum.IF_MATCH.toString(), sourceCuboidId 
== targetCuboidId);
-        this.metricsEvent.put(PropertyEnum.FILTER_MASK.toString(), filterMask);
-    }
-
-    public void setStats(long callCount, long callTimeSum, long callTimeMax, 
long skipCount, long scanCount,
-            long returnCount, long aggrCount, boolean ifSuccess, double 
weightPerHit) {
-        this.metricsEvent.put(PropertyEnum.CALL_COUNT.toString(), callCount);
-        this.metricsEvent.put(PropertyEnum.TIME_SUM.toString(), callTimeSum);
-        this.metricsEvent.put(PropertyEnum.TIME_MAX.toString(), callTimeMax);
-        this.metricsEvent.put(PropertyEnum.SKIP_COUNT.toString(), skipCount);
-        this.metricsEvent.put(PropertyEnum.SCAN_COUNT.toString(), scanCount);
-        this.metricsEvent.put(PropertyEnum.RETURN_COUNT.toString(), 
returnCount);
-        this.metricsEvent.put(PropertyEnum.AGGR_FILTER_COUNT.toString(), 
scanCount - returnCount);
-        this.metricsEvent.put(PropertyEnum.AGGR_COUNT.toString(), aggrCount);
-        this.metricsEvent.put(PropertyEnum.IF_SUCCESS.toString(), ifSuccess);
-        this.metricsEvent.put(PropertyEnum.WEIGHT_PER_HIT.toString(), 
weightPerHit);
-    }
-
-    public Object getProperty(String key) {
-        return this.metricsEvent.get(key);
-    }
-
-    public enum PropertyEnum {
-        PROJECT("PROJECT"), CUBE("CUBE_NAME"), SEGMENT("SEGMENT_NAME"), 
CUBOID_SOURCE("CUBOID_SOURCE"), CUBOID_TARGET(
-                "CUBOID_TARGET"), IF_MATCH("IF_MATCH"), 
FILTER_MASK("FILTER_MASK"), IF_SUCCESS("IF_SUCCESS"), //
-        TIME_SUM("STORAGE_CALL_TIME_SUM"), TIME_MAX("STORAGE_CALL_TIME_MAX"), 
WEIGHT_PER_HIT(
-                "WEIGHT_PER_HIT"), CALL_COUNT("STORAGE_CALL_COUNT"), 
SKIP_COUNT("STORAGE_COUNT_SKIP"), SCAN_COUNT(
-                        "STORAGE_COUNT_SCAN"), 
RETURN_COUNT("STORAGE_COUNT_RETURN"), AGGR_FILTER_COUNT(
-                                "STORAGE_COUNT_AGGREGATE_FILTER"), 
AGGR_COUNT("STORAGE_COUNT_AGGREGATE");
-
-        private final String propertyName;
-
-        PropertyEnum(String name) {
-            this.propertyName = name;
-        }
-
-        public static PropertyEnum getByName(String name) {
-            if (Strings.isNullOrEmpty(name)) {
-                return null;
-            }
-            for (PropertyEnum property : PropertyEnum.values()) {
-                if (property.propertyName.equals(name.toUpperCase())) {
-                    return property;
-                }
-            }
-
-            return null;
-        }
-
-        @Override
-        public String toString() {
-            return propertyName;
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/query/QueryRecordEventWrapper.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/query/QueryRecordEventWrapper.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/query/QueryRecordEventWrapper.java
deleted file mode 100644
index 8ea0222..0000000
--- 
a/core-metrics/src/main/java/org/apache/kylin/metrics/query/QueryRecordEventWrapper.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * 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.kylin.metrics.query;
-
-import org.apache.kylin.metrics.lib.impl.RecordEvent;
-import org.apache.kylin.metrics.lib.impl.RecordEventWrapper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Strings;
-
-public class QueryRecordEventWrapper extends RecordEventWrapper {
-
-    private static final Logger logger = 
LoggerFactory.getLogger(QueryRecordEventWrapper.class);
-
-    public QueryRecordEventWrapper(RecordEvent metricsEvent) {
-        super(metricsEvent);
-    }
-
-    public void setWrapper(long queryHashCode, String queryType, String 
projectName, String realizationName,
-            int realizationType, Throwable throwable) {
-        this.metricsEvent.put(PropertyEnum.ID_CODE.toString(), queryHashCode);
-        this.metricsEvent.put(PropertyEnum.TYPE.toString(), queryType);
-        this.metricsEvent.put(PropertyEnum.PROJECT.toString(), projectName);
-        this.metricsEvent.put(PropertyEnum.REALIZATION.toString(), 
realizationName);
-        this.metricsEvent.put(PropertyEnum.REALIZATION_TYPE.toString(), 
realizationType);
-        this.metricsEvent.put(PropertyEnum.EXCEPTION.toString(),
-                throwable == null ? "NULL" : throwable.getClass().getName());
-    }
-
-    public void setStats(long callTimeMs, long returnCountByCalcite, long 
returnCountByStorage) {
-        this.metricsEvent.put(PropertyEnum.TIME_COST.toString(), callTimeMs);
-        this.metricsEvent.put(PropertyEnum.CALCITE_RETURN_COUNT.toString(), 
returnCountByCalcite);
-        this.metricsEvent.put(PropertyEnum.STORAGE_RETURN_COUNT.toString(), 
returnCountByStorage);
-        long countAggrAndFilter = returnCountByStorage - returnCountByCalcite;
-        if (countAggrAndFilter < 0) {
-            countAggrAndFilter = 0;
-            logger.warn(returnCountByStorage + " rows returned by storage less 
than " + returnCountByCalcite
-                    + " rows returned by calcite");
-        }
-        this.metricsEvent.put(PropertyEnum.AGGR_FILTER_COUNT.toString(), 
countAggrAndFilter);
-    }
-
-    public enum PropertyEnum {
-        ID_CODE("QUERY_HASH_CODE"), TYPE("QUERY_TYPE"), PROJECT("PROJECT"), 
REALIZATION(
-                "REALIZATION"), REALIZATION_TYPE("REALIZATION_TYPE"), 
EXCEPTION("EXCEPTION"), //
-        TIME_COST("QUERY_TIME_COST"), 
CALCITE_RETURN_COUNT("CALCITE_COUNT_RETURN"), STORAGE_RETURN_COUNT(
-                "STORAGE_COUNT_RETURN"), 
AGGR_FILTER_COUNT("CALCITE_COUNT_AGGREGATE_FILTER");
-
-        private final String propertyName;
-
-        PropertyEnum(String name) {
-            this.propertyName = name;
-        }
-
-        public static PropertyEnum getByName(String name) {
-            if (Strings.isNullOrEmpty(name)) {
-                return null;
-            }
-            for (PropertyEnum property : PropertyEnum.values()) {
-                if (property.propertyName.equals(name.toUpperCase())) {
-                    return property;
-                }
-            }
-
-            return null;
-        }
-
-        @Override
-        public String toString() {
-            return propertyName;
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/core-metrics/src/main/java/org/apache/kylin/metrics/query/RPCRecordEventWrapper.java
----------------------------------------------------------------------
diff --git 
a/core-metrics/src/main/java/org/apache/kylin/metrics/query/RPCRecordEventWrapper.java
 
b/core-metrics/src/main/java/org/apache/kylin/metrics/query/RPCRecordEventWrapper.java
deleted file mode 100644
index abcbb61..0000000
--- 
a/core-metrics/src/main/java/org/apache/kylin/metrics/query/RPCRecordEventWrapper.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * 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.kylin.metrics.query;
-
-import org.apache.kylin.metrics.lib.impl.RecordEvent;
-import org.apache.kylin.metrics.lib.impl.RecordEventWrapper;
-
-import com.google.common.base.Strings;
-
-public class RPCRecordEventWrapper extends RecordEventWrapper {
-
-    public RPCRecordEventWrapper(RecordEvent metricsEvent) {
-        super(metricsEvent);
-    }
-
-    public void setWrapper(String projectName, String realizationName, String 
rpcServer, Throwable throwable) {
-        this.metricsEvent.put(PropertyEnum.PROJECT.toString(), projectName);
-        this.metricsEvent.put(PropertyEnum.REALIZATION.toString(), 
realizationName);
-        this.metricsEvent.put(PropertyEnum.RPC_SERVER.toString(), rpcServer);
-        this.metricsEvent.put(PropertyEnum.EXCEPTION.toString(),
-                throwable == null ? "NULL" : throwable.getClass().getName());
-    }
-
-    public void setStats(long callTimeMs, long skipCount, long scanCount, long 
returnCount, long aggrCount) {
-        this.metricsEvent.put(PropertyEnum.CALL_TIME.toString(), callTimeMs);
-        this.metricsEvent.put(PropertyEnum.SKIP_COUNT.toString(), skipCount); 
//Number of skips on region servers based on region meta or fuzzy filter
-        this.metricsEvent.put(PropertyEnum.SCAN_COUNT.toString(), scanCount); 
//Count scanned by region server
-        this.metricsEvent.put(PropertyEnum.RETURN_COUNT.toString(), 
returnCount);//Count returned by region server
-        this.metricsEvent.put(PropertyEnum.AGGR_FILTER_COUNT.toString(), 
scanCount - returnCount); //Count filtered & aggregated by coprocessor
-        this.metricsEvent.put(PropertyEnum.AGGR_COUNT.toString(), aggrCount); 
//Count aggregated by coprocessor
-    }
-
-    public enum PropertyEnum {
-        PROJECT("PROJECT"), REALIZATION("REALIZATION"), 
RPC_SERVER("RPC_SERVER"), EXCEPTION("EXCEPTION"), //
-        CALL_TIME("CALL_TIME"), SKIP_COUNT("COUNT_SKIP"), 
SCAN_COUNT("COUNT_SCAN"), RETURN_COUNT(
-                "COUNT_RETURN"), AGGR_FILTER_COUNT("COUNT_AGGREGATE_FILTER"), 
AGGR_COUNT("COUNT_AGGREGATE");
-
-        private final String propertyName;
-
-        PropertyEnum(String name) {
-            this.propertyName = name;
-        }
-
-        public static PropertyEnum getByName(String name) {
-            if (Strings.isNullOrEmpty(name)) {
-                return null;
-            }
-            for (PropertyEnum property : PropertyEnum.values()) {
-                if (property.propertyName.equals(name.toUpperCase())) {
-                    return property;
-                }
-            }
-
-            return null;
-        }
-
-        @Override
-        public String toString() {
-            return propertyName;
-        }
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
----------------------------------------------------------------------
diff --git a/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java 
b/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
index c4b6e12..79f40ae 100644
--- a/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
+++ b/engine-mr/src/main/java/org/apache/kylin/engine/mr/CubingJob.java
@@ -46,12 +46,12 @@ import org.apache.kylin.job.execution.ExecutableContext;
 import org.apache.kylin.job.execution.ExecutableState;
 import org.apache.kylin.job.execution.ExecuteResult;
 import org.apache.kylin.job.execution.Output;
+import org.apache.kylin.job.metrics.JobMetricsFacade;
 import org.apache.kylin.metadata.project.ProjectInstance;
 import org.apache.kylin.metadata.project.ProjectManager;
 import org.apache.kylin.metrics.MetricsManager;
-import org.apache.kylin.metrics.job.ExceptionRecordEventWrapper;
-import org.apache.kylin.metrics.job.JobRecordEventWrapper;
 import org.apache.kylin.metrics.lib.impl.RecordEvent;
+import org.apache.kylin.metrics.lib.impl.TimedRecordEvent;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -249,37 +249,49 @@ public class CubingJob extends DefaultChainedExecutable {
          * report job related metrics
          */
         if (state == ExecutableState.SUCCEED) {
-            JobRecordEventWrapper jobRecordEventWrapper = new 
JobRecordEventWrapper(
-                    new 
RecordEvent(KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectJob()));
-            
jobRecordEventWrapper.setWrapper(ProjectInstance.getNormalizedProjectName(getProjectName()),
+            RecordEvent metricsEvent = new TimedRecordEvent(
+                    
KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectJob());
+            JobMetricsFacade.setJobWrapper(metricsEvent, //
+                    ProjectInstance.getNormalizedProjectName(getProjectName()),
                     CubingExecutableUtil.getCubeName(getParams()), getId(), 
getJobType(),
                     getAlgorithm() == null ? "NULL" : 
getAlgorithm().toString());
+
             long tableSize = findSourceSizeBytes();
             long buildDuration = getDuration();
             long waitResourceTime = getMapReduceWaitTime();
-            jobRecordEventWrapper.setStats(tableSize, findCubeSizeBytes(), 
buildDuration, waitResourceTime,
+            JobMetricsFacade.setJobStats(metricsEvent, //
+                    tableSize, findCubeSizeBytes(), buildDuration, 
waitResourceTime,
                     getPerBytesTimeCost(tableSize, buildDuration - 
waitResourceTime));
+            long dColumnDistinct = 0L;
+            long dDictBuilding = 0L;
+            long dCubingInmem = 0L;
+            long dHfileConvert = 0L;
             if (CubingJobTypeEnum.getByName(getJobType()) == 
CubingJobTypeEnum.BUILD) {
-                jobRecordEventWrapper.setStepStats(
-                        
getTaskByName(ExecutableConstants.STEP_NAME_FACT_DISTINCT_COLUMNS).getDuration(),
 //
-                        
getTaskByName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY).getDuration(), //
-                        
getTaskByName(ExecutableConstants.STEP_NAME_BUILD_IN_MEM_CUBE).getDuration(), //
-                        
getTaskByName(ExecutableConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE).getDuration());
+                dColumnDistinct = 
getTaskByName(ExecutableConstants.STEP_NAME_FACT_DISTINCT_COLUMNS).getDuration();
+                dDictBuilding = 
getTaskByName(ExecutableConstants.STEP_NAME_BUILD_DICTIONARY).getDuration();
+                dCubingInmem = 
getTaskByName(ExecutableConstants.STEP_NAME_BUILD_IN_MEM_CUBE).getDuration();
+                dHfileConvert = 
getTaskByName(ExecutableConstants.STEP_NAME_CONVERT_CUBOID_TO_HFILE).getDuration();
             }
-            
MetricsManager.getInstance().update(jobRecordEventWrapper.getMetricsRecord());
+            JobMetricsFacade.setJobStepStats(metricsEvent, //
+                    dColumnDistinct, dDictBuilding, dCubingInmem, 
dHfileConvert);
+
+            MetricsManager.getInstance().update(metricsEvent);
         } else if (state == ExecutableState.ERROR) {
-            ExceptionRecordEventWrapper exceptionRecordEventWrapper = new 
ExceptionRecordEventWrapper(
-                    new 
RecordEvent(KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectJobException()));
-            
exceptionRecordEventWrapper.setWrapper(ProjectInstance.getNormalizedProjectName(getProjectName()),
+            RecordEvent metricsEvent = new TimedRecordEvent(
+                    
KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectJobException());
+
+            Class throwable = result.getThrowable() != null ? 
result.getThrowable().getClass() : Exception.class;
+            JobMetricsFacade.setJobExceptionWrapper(metricsEvent, //
+                    ProjectInstance.getNormalizedProjectName(getProjectName()),
                     CubingExecutableUtil.getCubeName(getParams()), getId(), 
getJobType(),
-                    getAlgorithm() == null ? "NULL" : 
getAlgorithm().toString(),
-                    result.getThrowable() != null ? 
result.getThrowable().getClass() : Exception.class);
-            
MetricsManager.getInstance().update(exceptionRecordEventWrapper.getMetricsRecord());
+                    getAlgorithm() == null ? "NULL" : 
getAlgorithm().toString(), throwable);
+
+            MetricsManager.getInstance().update(metricsEvent);
         }
 
     }
 
-    private double getPerBytesTimeCost(long size, long timeCost) {
+    private static double getPerBytesTimeCost(long size, long timeCost) {
         if (size <= 0) {
             return 0;
         }

http://git-wip-us.apache.org/repos/asf/kylin/blob/02b67ceb/server-base/src/main/java/org/apache/kylin/rest/metrics/QueryMetricsFacade.java
----------------------------------------------------------------------
diff --git 
a/server-base/src/main/java/org/apache/kylin/rest/metrics/QueryMetricsFacade.java
 
b/server-base/src/main/java/org/apache/kylin/rest/metrics/QueryMetricsFacade.java
index 8b58382..67ef203 100644
--- 
a/server-base/src/main/java/org/apache/kylin/rest/metrics/QueryMetricsFacade.java
+++ 
b/server-base/src/main/java/org/apache/kylin/rest/metrics/QueryMetricsFacade.java
@@ -30,9 +30,10 @@ import org.apache.kylin.common.KylinConfig;
 import org.apache.kylin.common.QueryContext;
 import org.apache.kylin.metrics.MetricsManager;
 import org.apache.kylin.metrics.lib.impl.RecordEvent;
-import org.apache.kylin.metrics.query.CubeSegmentRecordEventWrapper;
-import org.apache.kylin.metrics.query.QueryRecordEventWrapper;
-import org.apache.kylin.metrics.query.RPCRecordEventWrapper;
+import org.apache.kylin.metrics.lib.impl.TimedRecordEvent;
+import org.apache.kylin.metrics.property.QueryCubePropertyEnum;
+import org.apache.kylin.metrics.property.QueryPropertyEnum;
+import org.apache.kylin.metrics.property.QueryRPCPropertyEnum;
 import org.apache.kylin.rest.request.SQLRequest;
 import org.apache.kylin.rest.response.SQLResponse;
 import org.slf4j.Logger;
@@ -84,53 +85,126 @@ public class QueryMetricsFacade {
          */
         final QueryContext.QueryStatisticsResult queryStatisticsResult = 
sqlResponse.getQueryStatistics();
         for (QueryContext.RPCStatistics entry : 
queryStatisticsResult.getRpcStatisticsList()) {
-            RPCRecordEventWrapper rpcMetricsEventWrapper = new 
RPCRecordEventWrapper(
-                    new 
RecordEvent(KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectQueryRpcCall()));
-            rpcMetricsEventWrapper.setWrapper(sqlRequest.getProject(), 
entry.getRealizationName(), entry.getRpcServer(),
+            RecordEvent rpcMetricsEvent = new TimedRecordEvent(
+                    
KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectQueryRpcCall());
+            setRPCWrapper(rpcMetricsEvent, //
+                    sqlRequest.getProject(), entry.getRealizationName(), 
entry.getRpcServer(),
                     entry.getException());
-            rpcMetricsEventWrapper.setStats(entry.getCallTimeMs(), 
entry.getSkippedRows(), entry.getScannedRows(),
+            setRPCStats(rpcMetricsEvent, //
+                    entry.getCallTimeMs(), entry.getSkippedRows(), 
entry.getScannedRows(),
                     entry.getReturnedRows(), entry.getAggregatedRows());
             //For update rpc level related metrics
-            
MetricsManager.getInstance().update(rpcMetricsEventWrapper.getMetricsRecord());
+            MetricsManager.getInstance().update(rpcMetricsEvent);
         }
         long sqlHashCode = getSqlHashCode(sqlRequest.getSql());
         for (QueryContext.CubeSegmentStatisticsResult contextEntry : 
queryStatisticsResult
                 .getCubeSegmentStatisticsResultList()) {
-            QueryRecordEventWrapper queryMetricsEventWrapper = new 
QueryRecordEventWrapper(
-                    new 
RecordEvent(KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectQuery()));
-            queryMetricsEventWrapper.setWrapper(sqlHashCode,
-                    sqlResponse.isStorageCacheUsed() ? "CACHE" : 
contextEntry.getQueryType(), sqlRequest.getProject(),
-                    contextEntry.getRealization(), 
contextEntry.getRealizationType(), sqlResponse.getThrowable());
+            RecordEvent queryMetricsEvent = new TimedRecordEvent(
+                    
KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectQuery());
+            setQueryWrapper(queryMetricsEvent, //
+                    sqlHashCode, sqlResponse.isStorageCacheUsed() ? "CACHE" : 
contextEntry.getQueryType(),
+                    sqlRequest.getProject(), contextEntry.getRealization(), 
contextEntry.getRealizationType(),
+                    sqlResponse.getThrowable());
 
             long totalStorageReturnCount = 0L;
             for (Map<String, QueryContext.CubeSegmentStatistics> cubeEntry : 
contextEntry.getCubeSegmentStatisticsMap()
                     .values()) {
                 for (QueryContext.CubeSegmentStatistics segmentEntry : 
cubeEntry.values()) {
-                    CubeSegmentRecordEventWrapper 
cubeSegmentMetricsEventWrapper = new CubeSegmentRecordEventWrapper(
-                            new 
RecordEvent(KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectQueryCube()));
+                    RecordEvent cubeSegmentMetricsEvent = new TimedRecordEvent(
+                            
KylinConfig.getInstanceFromEnv().getKylinMetricsSubjectQueryCube());
 
-                    
cubeSegmentMetricsEventWrapper.setWrapper(sqlRequest.getProject(), 
segmentEntry.getCubeName(),
+                    setCubeWrapper(cubeSegmentMetricsEvent, //
+                            sqlRequest.getProject(), 
segmentEntry.getCubeName(),
                             segmentEntry.getSegmentName(), 
segmentEntry.getSourceCuboidId(),
                             segmentEntry.getTargetCuboidId(), 
segmentEntry.getFilterMask());
 
-                    
cubeSegmentMetricsEventWrapper.setStats(segmentEntry.getCallCount(), 
segmentEntry.getCallTimeSum(),
-                            segmentEntry.getCallTimeMax(), 
segmentEntry.getStorageSkippedRows(),
-                            segmentEntry.getStorageScannedRows(), 
segmentEntry.getStorageReturnedRows(),
-                            segmentEntry.getStorageAggregatedRows(), 
segmentEntry.isIfSuccess(),
-                            1.0 / cubeEntry.size());
+                    setCubeStats(cubeSegmentMetricsEvent, //
+                            segmentEntry.getCallCount(), 
segmentEntry.getCallTimeSum(), segmentEntry.getCallTimeMax(),
+                            segmentEntry.getStorageSkippedRows(), 
segmentEntry.getStorageScannedRows(),
+                            segmentEntry.getStorageReturnedRows(), 
segmentEntry.getStorageAggregatedRows(),
+                            segmentEntry.isIfSuccess(), 1.0 / 
cubeEntry.size());
 
                     totalStorageReturnCount += 
segmentEntry.getStorageReturnedRows();
                     //For update cube segment level related query metrics
-                    
MetricsManager.getInstance().update(cubeSegmentMetricsEventWrapper.getMetricsRecord());
+                    
MetricsManager.getInstance().update(cubeSegmentMetricsEvent);
                 }
             }
-            queryMetricsEventWrapper.setStats(sqlResponse.getDuration(), 
sqlResponse.getResults().size(),
-                    totalStorageReturnCount);
+            setQueryStats(queryMetricsEvent, //
+                    sqlResponse.getDuration(), 
sqlResponse.getResults().size(), totalStorageReturnCount);
             //For update query level metrics
-            
MetricsManager.getInstance().update(queryMetricsEventWrapper.getMetricsRecord());
+            MetricsManager.getInstance().update(queryMetricsEvent);
         }
     }
 
+    public static void setRPCWrapper(RecordEvent metricsEvent, String 
projectName, String realizationName,
+            String rpcServer, Throwable throwable) {
+        metricsEvent.put(QueryRPCPropertyEnum.PROJECT.toString(), projectName);
+        metricsEvent.put(QueryRPCPropertyEnum.REALIZATION.toString(), 
realizationName);
+        metricsEvent.put(QueryRPCPropertyEnum.RPC_SERVER.toString(), 
rpcServer);
+        metricsEvent.put(QueryRPCPropertyEnum.EXCEPTION.toString(),
+                throwable == null ? "NULL" : throwable.getClass().getName());
+    }
+
+    public static void setRPCStats(RecordEvent metricsEvent, long callTimeMs, 
long skipCount, long scanCount,
+            long returnCount, long aggrCount) {
+        metricsEvent.put(QueryRPCPropertyEnum.CALL_TIME.toString(), 
callTimeMs);
+        metricsEvent.put(QueryRPCPropertyEnum.SKIP_COUNT.toString(), 
skipCount); //Number of skips on region servers based on region meta or fuzzy 
filter
+        metricsEvent.put(QueryRPCPropertyEnum.SCAN_COUNT.toString(), 
scanCount); //Count scanned by region server
+        metricsEvent.put(QueryRPCPropertyEnum.RETURN_COUNT.toString(), 
returnCount);//Count returned by region server
+        metricsEvent.put(QueryRPCPropertyEnum.AGGR_FILTER_COUNT.toString(), 
scanCount - returnCount); //Count filtered & aggregated by coprocessor
+        metricsEvent.put(QueryRPCPropertyEnum.AGGR_COUNT.toString(), 
aggrCount); //Count aggregated by coprocessor
+    }
+
+    public static void setCubeWrapper(RecordEvent metricsEvent, String 
projectName, String cubeName, String segmentName,
+            long sourceCuboidId, long targetCuboidId, long filterMask) {
+        metricsEvent.put(QueryCubePropertyEnum.PROJECT.toString(), 
projectName);
+        metricsEvent.put(QueryCubePropertyEnum.CUBE.toString(), cubeName);
+        metricsEvent.put(QueryCubePropertyEnum.SEGMENT.toString(), 
segmentName);
+        metricsEvent.put(QueryCubePropertyEnum.CUBOID_SOURCE.toString(), 
sourceCuboidId);
+        metricsEvent.put(QueryCubePropertyEnum.CUBOID_TARGET.toString(), 
targetCuboidId);
+        metricsEvent.put(QueryCubePropertyEnum.IF_MATCH.toString(), 
sourceCuboidId == targetCuboidId);
+        metricsEvent.put(QueryCubePropertyEnum.FILTER_MASK.toString(), 
filterMask);
+    }
+
+    public static void setCubeStats(RecordEvent metricsEvent, long callCount, 
long callTimeSum, long callTimeMax,
+            long skipCount, long scanCount, long returnCount, long aggrCount, 
boolean ifSuccess, double weightPerHit) {
+        metricsEvent.put(QueryCubePropertyEnum.CALL_COUNT.toString(), 
callCount);
+        metricsEvent.put(QueryCubePropertyEnum.TIME_SUM.toString(), 
callTimeSum);
+        metricsEvent.put(QueryCubePropertyEnum.TIME_MAX.toString(), 
callTimeMax);
+        metricsEvent.put(QueryCubePropertyEnum.SKIP_COUNT.toString(), 
skipCount);
+        metricsEvent.put(QueryCubePropertyEnum.SCAN_COUNT.toString(), 
scanCount);
+        metricsEvent.put(QueryCubePropertyEnum.RETURN_COUNT.toString(), 
returnCount);
+        metricsEvent.put(QueryCubePropertyEnum.AGGR_FILTER_COUNT.toString(), 
scanCount - returnCount);
+        metricsEvent.put(QueryCubePropertyEnum.AGGR_COUNT.toString(), 
aggrCount);
+        metricsEvent.put(QueryCubePropertyEnum.IF_SUCCESS.toString(), 
ifSuccess);
+        metricsEvent.put(QueryCubePropertyEnum.WEIGHT_PER_HIT.toString(), 
weightPerHit);
+    }
+
+    public static void setQueryWrapper(RecordEvent metricsEvent, long 
queryHashCode, String queryType,
+            String projectName, String realizationName, int realizationType, 
Throwable throwable) {
+        metricsEvent.put(QueryPropertyEnum.ID_CODE.toString(), queryHashCode);
+        metricsEvent.put(QueryPropertyEnum.TYPE.toString(), queryType);
+        metricsEvent.put(QueryPropertyEnum.PROJECT.toString(), projectName);
+        metricsEvent.put(QueryPropertyEnum.REALIZATION.toString(), 
realizationName);
+        metricsEvent.put(QueryPropertyEnum.REALIZATION_TYPE.toString(), 
realizationType);
+        metricsEvent.put(QueryPropertyEnum.EXCEPTION.toString(),
+                throwable == null ? "NULL" : throwable.getClass().getName());
+    }
+
+    public static void setQueryStats(RecordEvent metricsEvent, long 
callTimeMs, long returnCountByCalcite,
+            long returnCountByStorage) {
+        metricsEvent.put(QueryPropertyEnum.TIME_COST.toString(), callTimeMs);
+        metricsEvent.put(QueryPropertyEnum.CALCITE_RETURN_COUNT.toString(), 
returnCountByCalcite);
+        metricsEvent.put(QueryPropertyEnum.STORAGE_RETURN_COUNT.toString(), 
returnCountByStorage);
+        long countAggrAndFilter = returnCountByStorage - returnCountByCalcite;
+        if (countAggrAndFilter < 0) {
+            countAggrAndFilter = 0;
+            logger.warn(returnCountByStorage + " rows returned by storage less 
than " + returnCountByCalcite
+                    + " rows returned by calcite");
+        }
+        metricsEvent.put(QueryPropertyEnum.AGGR_FILTER_COUNT.toString(), 
countAggrAndFilter);
+    }
+
     private static void update(QueryMetrics queryMetrics, SQLResponse 
sqlResponse) {
         try {
             incrQueryCount(queryMetrics, sqlResponse);

Reply via email to