You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@submarine.apache.org by GitBox <gi...@apache.org> on 2020/06/14 16:52:43 UTC

[GitHub] [submarine] JohnTing opened a new pull request #311: Submarine-500. Add metric, param instance to the submarine-server rest

JohnTing opened a new pull request #311:
URL: https://github.com/apache/submarine/pull/311


   ### What is this PR for?
   Add  metric, param instance from submarine/store/database/models.py  to the submarine-server rest api
   
   A few sentences describing the overall goals of the pull request's commits.
   First time? Check out the contributing guide - https://submarine.apache.org/contribution/contributions.html
   
   
   ### What type of PR is it?
   Feature 
   
   ### Todos
   * [ ] - metric rest api
   * [ ] - param rest api
   * [ ] - database
   
   ### What is the Jira issue?
   https://issues.apache.org/jira/projects/SUBMARINE/issues/SUBMARINE-500
   
   ### How should this be tested?
   https://travis-ci.org/github/JohnTing/submarine/builds/698241108
   
   ### Screenshots (if appropriate)
   ![image](https://user-images.githubusercontent.com/19265751/84598565-37a75380-ae9e-11ea-895d-355ee0cd66be.png)
   ![image](https://user-images.githubusercontent.com/19265751/84598593-82c16680-ae9e-11ea-8220-68e5244b5b0d.png)
   
   ![image](https://user-images.githubusercontent.com/19265751/84598585-6d4c3c80-ae9e-11ea-8ac9-2bdaa55b3883.png)
   ![image](https://user-images.githubusercontent.com/19265751/84598591-789f6800-ae9e-11ea-90a4-475f0c5e5737.png)
   
   ### Questions:
   * Does the licenses files need update? No
   * Is there breaking changes for older versions? No
   * Does this needs documentation? Yes/No
   


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

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



[GitHub] [submarine] liuxunorg commented on a change in pull request #311: SUBMARINE-500. Add metric, param instance to the submarine-server rest

Posted by GitBox <gi...@apache.org>.
liuxunorg commented on a change in pull request #311:
URL: https://github.com/apache/submarine/pull/311#discussion_r442578358



##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getMetric(@QueryParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();

Review comment:
       `Response.Status.OK` means the rest request status was successful.
   `success(true)` or `success(flse)` indicates success or failure of the operation.
   So, Here should be changed to
   ```
   return new JsonResponse.Builder<String>(Response.Status.OK).success(false).build();
   ```
   
   Other places in this PR also need to be revised.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/database/service/MetricService.java
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import org.apache.submarine.server.workbench.database.MyBatisUtil;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.mappers.MetricMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import org.apache.ibatis.session.SqlSession;
+
+public class MetricService {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricService.class);
+
+  public MetricService() {
+    
+  }
+
+  

Review comment:
       Only need a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getMetric(@QueryParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+

Review comment:
       delete this blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();

Review comment:
       Don't use `e.printStackTrace()` print log.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getParam(@QueryParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();

Review comment:
       Don't use e.printStackTrace() print log.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getMetric(@QueryParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.update(metric);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @POST
+  @Path("/selective")
+  @SubmarineApi
+  public Response selectByPrimaryKeySelective(Metric metric) {
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+

Review comment:
       delete this blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/database/entity/Metric.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.submarine.server.workbench.database.entity;
+
+import java.math.BigInteger;
+
+/*
+# +-------+----------+--------------+---------------+------+--------+------------------+
+# | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+# +-------+----------+--------------+---------------+------+--------+------------------+
+# | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+# | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+# | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+# | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+# | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+# +-------+----------+--------------+---------------+------+--------+------------------+
+*/

Review comment:
       Good job!

##########
File path: submarine-server/server-core/src/test/java/org/apache/submarine/server/workbench/database/service/ParamServiceTest.java
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.junit.After;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import static junit.framework.TestCase.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+public class ParamServiceTest {
+  private static final Logger LOG = LoggerFactory.getLogger(ParamServiceTest.class);
+  ParamService paramService = new ParamService();
+
+  @After
+  public void removeAllRecord() throws Exception {
+    List<Param> paramList = paramService.selectAll();
+    LOG.info("paramList.size():{}", paramList.size());
+    for (Param param : paramList) {
+      paramService.deleteById(param.getId());
+    }
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    Param param = new Param();
+    param.setParamKey("score");
+    param.setValue("199");
+    param.setWorkerIndex("worker-1");
+    param.setJobName("application_123651651");
+    boolean result = paramService.insert(param);
+    assertNotEquals(result, -1);

Review comment:
       change to `assertTrue(result);`

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getMetric(@QueryParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();

Review comment:
       Change to `.success(false).build()`.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getMetric(@QueryParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();

Review comment:
       Change to `.success(false).build()`.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getMetric(@QueryParam("id") String id) {

Review comment:
       Maybe changing id to path variable would be better.
   like., 
   ```
   @Path("/{id}")
   @SubmarineApi
   public Response getMetric(@PahtParam("id") String id) {
   ```

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getParam(@QueryParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();

Review comment:
       Don't use e.printStackTrace() print log.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getParam(@QueryParam("id") String id) {

Review comment:
       Maybe changing `id` to path variable would be better.
   like., 
   ```
   @Path("/{id}")
   @SubmarineApi
   public Response getParam(@PahtParam("id") String id) {
   ```

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getMetric(@QueryParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.update(metric);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @POST
+  @Path("/selective")
+  @SubmarineApi
+  public Response selectByPrimaryKeySelective(Metric metric) {
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+

Review comment:
       delete this blank lline.

##########
File path: submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/MetricMapper.xml
##########
@@ -0,0 +1,108 @@
+<?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.
+-->
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.submarine.server.workbench.database.mappers.MetricMapper">
+  <resultMap id="BaseEntityResultMap" type="org.apache.submarine.server.workbench.database.entity.BaseEntity">
+    <id property="id" column="id"/>
+  </resultMap>
+
+  <resultMap id="resultMap" type="org.apache.submarine.server.workbench.database.entity.Metric" extends="BaseEntityResultMap">
+    <result column="metric_key" jdbcType="VARCHAR" property="metricKey" />
+    <result column="value" jdbcType="FLOAT" property="value" />
+    <result column="worker_index" jdbcType="VARCHAR" property="workerIndex" />
+    <result column="timestamp" jdbcType="BIGINT" property="timestamp" />
+    <result column="step" jdbcType="INTEGER" property="step" />
+    <result column="is_nan" jdbcType="INTEGER" property="isNan" />
+    <result column="job_name" jdbcType="VARCHAR" property="jobName" />
+  </resultMap>
+
+  <sql id="Base_Column_List">
+    id, metric_key, value, worker_index, timestamp, step, is_nan, job_name
+  </sql>
+
+  <select id="selectAll" parameterType="java.lang.String" resultMap="resultMap">
+    select
+    <include refid="Base_Column_List" />
+    from metric
+    where 1 = 1
+  </select>
+
+  <select id="selectById" parameterType="java.lang.String" resultMap="resultMap">
+    select
+    <include refid="Base_Column_List" />
+    from metric
+    where id = #{id,jdbcType=VARCHAR}
+  </select>
+
+  <delete id="deleteById" parameterType="java.lang.String">
+    delete from metric
+    where id = #{id,jdbcType=VARCHAR}
+  </delete>
+
+  <insert id="insert" parameterType="org.apache.submarine.server.workbench.database.entity.Metric"
+          useGeneratedKeys="true" keyProperty="id">
+    insert into metric (metric_key, value, worker_index, timestamp,
+      step, is_nan, job_name)
+    values (#{metricKey,jdbcType=VARCHAR}, 
+      #{value,jdbcType=FLOAT},
+      #{workerIndex,jdbcType=VARCHAR}, 
+      #{timestamp,jdbcType=BIGINT}, 
+      #{step,jdbcType=INTEGER},
+      #{isNan,jdbcType=INTEGER}, 
+      #{jobName,jdbcType=VARCHAR})
+  </insert>
+  
+  <update id="update" parameterType="org.apache.submarine.server.workbench.database.entity.Metric">
+    update metric
+    set metric_key = #{metricKey,jdbcType=VARCHAR},
+      value = #{value,jdbcType=FLOAT},
+      worker_index = #{workerIndex,jdbcType=VARCHAR},
+      timestamp = #{timestamp,jdbcType=BIGINT},
+      step = #{step,jdbcType=INTEGER},
+      is_nan = #{isNan,jdbcType=INTEGER},
+      job_name = #{jobName,jdbcType=VARCHAR}
+    where id = #{id,jdbcType=VARCHAR}
+  </update>
+  
+
+

Review comment:
       Keep only one blank line

##########
File path: submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/MetricMapper.xml
##########
@@ -0,0 +1,108 @@
+<?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.
+-->
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.submarine.server.workbench.database.mappers.MetricMapper">
+  <resultMap id="BaseEntityResultMap" type="org.apache.submarine.server.workbench.database.entity.BaseEntity">
+    <id property="id" column="id"/>
+  </resultMap>
+
+  <resultMap id="resultMap" type="org.apache.submarine.server.workbench.database.entity.Metric" extends="BaseEntityResultMap">
+    <result column="metric_key" jdbcType="VARCHAR" property="metricKey" />
+    <result column="value" jdbcType="FLOAT" property="value" />
+    <result column="worker_index" jdbcType="VARCHAR" property="workerIndex" />
+    <result column="timestamp" jdbcType="BIGINT" property="timestamp" />
+    <result column="step" jdbcType="INTEGER" property="step" />
+    <result column="is_nan" jdbcType="INTEGER" property="isNan" />
+    <result column="job_name" jdbcType="VARCHAR" property="jobName" />
+  </resultMap>
+
+  <sql id="Base_Column_List">
+    id, metric_key, value, worker_index, timestamp, step, is_nan, job_name
+  </sql>
+
+  <select id="selectAll" parameterType="java.lang.String" resultMap="resultMap">
+    select
+    <include refid="Base_Column_List" />
+    from metric
+    where 1 = 1
+  </select>
+
+  <select id="selectById" parameterType="java.lang.String" resultMap="resultMap">
+    select
+    <include refid="Base_Column_List" />
+    from metric
+    where id = #{id,jdbcType=VARCHAR}
+  </select>
+
+  <delete id="deleteById" parameterType="java.lang.String">
+    delete from metric
+    where id = #{id,jdbcType=VARCHAR}
+  </delete>
+
+  <insert id="insert" parameterType="org.apache.submarine.server.workbench.database.entity.Metric"
+          useGeneratedKeys="true" keyProperty="id">
+    insert into metric (metric_key, value, worker_index, timestamp,
+      step, is_nan, job_name)
+    values (#{metricKey,jdbcType=VARCHAR}, 
+      #{value,jdbcType=FLOAT},
+      #{workerIndex,jdbcType=VARCHAR}, 
+      #{timestamp,jdbcType=BIGINT}, 
+      #{step,jdbcType=INTEGER},
+      #{isNan,jdbcType=INTEGER}, 
+      #{jobName,jdbcType=VARCHAR})
+  </insert>
+  
+  <update id="update" parameterType="org.apache.submarine.server.workbench.database.entity.Metric">
+    update metric
+    set metric_key = #{metricKey,jdbcType=VARCHAR},
+      value = #{value,jdbcType=FLOAT},
+      worker_index = #{workerIndex,jdbcType=VARCHAR},
+      timestamp = #{timestamp,jdbcType=BIGINT},
+      step = #{step,jdbcType=INTEGER},
+      is_nan = #{isNan,jdbcType=INTEGER},
+      job_name = #{jobName,jdbcType=VARCHAR}
+    where id = #{id,jdbcType=VARCHAR}
+  </update>
+  
+
+
+    <select id="selectByPrimaryKeySelective" parameterType="java.lang.String" resultMap="resultMap">

Review comment:
       2 spaces need to be indented

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,183 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getMetric(@QueryParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.update(metric);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @POST
+  @Path("/selective")
+  @SubmarineApi
+  public Response selectByPrimaryKeySelective(Metric metric) {
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+

Review comment:
       delete this blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,178 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/")
+  @SubmarineApi
+  public Response getParam(@QueryParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+    } catch (Exception e) {
+
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteParam(@QueryParam("id") String id) {
+    LOG.info("deleteParam ({})", id);
+    boolean result = false;
+    try {
+      result = paramService.deleteById(id);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();
+      return new JsonResponse.Builder<String>(Response.Status.OK).success(true).result("error").build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putParam(Param param) {
+    LOG.info("putParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.update(param);
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      e.printStackTrace();

Review comment:
       Don't use e.printStackTrace() print log.

##########
File path: submarine-server/server-core/src/test/java/org/apache/submarine/server/workbench/database/service/MetricServiceTest.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.junit.After;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import static junit.framework.TestCase.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+
+public class MetricServiceTest {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricServiceTest.class);
+  MetricService metricService = new MetricService();
+
+  @After
+  public void removeAllRecord() throws Exception {
+    List<Metric> metricList = metricService.selectAll();
+    LOG.info("jobList.size():{}", metricList.size());
+    for (Metric metric : metricList) {
+      metricService.deleteById(metric.getId());
+    }
+  }
+
+  @Test
+  public void testSelect() throws Exception {
+    Metric metric = new Metric();
+    metric.setMetricKey("score");
+    metric.setValue((float) 0.666667);
+    metric.setWorkerIndex("worker-1");
+    metric.setTimestamp(new BigInteger("1569139525097"));
+    metric.setStep(0);
+    metric.setIsNan(0);
+    metric.setJobName("application_1234");
+    boolean result = metricService.insert(metric);
+    assertNotEquals(result, -1);

Review comment:
       result is boolean type, It should be compared to `true` or `false`, not to `-1`
   
   assertTrue(result);




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

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



[GitHub] [submarine] pingsutw commented on a change in pull request #311: SUBMARINE-500. Add metric, param instance to the submarine-server rest

Posted by GitBox <gi...@apache.org>.
pingsutw commented on a change in pull request #311:
URL: https://github.com/apache/submarine/pull/311#discussion_r445006664



##########
File path: docs/database/submarine.sql
##########
@@ -219,3 +219,32 @@ CREATE TABLE `job` (
   `update_time` datetime default NULL COMMENT 'last update time',
   PRIMARY KEY  (`id`)
   ) ENGINE=InnoDB DEFAULT CHARSET=utf8;
+  
+-- ----------------------------
+-- Table structure for metric
+-- ----------------------------
+DROP TABLE IF EXISTS `metric`;
+CREATE TABLE `metric` (

Review comment:
       ```suggestion
   DROP TABLE IF EXISTS `metrics`;
   CREATE TABLE `metrics` (
   ```

##########
File path: docs/database/submarine.sql
##########
@@ -219,3 +219,32 @@ CREATE TABLE `job` (
   `update_time` datetime default NULL COMMENT 'last update time',
   PRIMARY KEY  (`id`)
   ) ENGINE=InnoDB DEFAULT CHARSET=utf8;
+  
+-- ----------------------------
+-- Table structure for metric
+-- ----------------------------
+DROP TABLE IF EXISTS `metric`;
+CREATE TABLE `metric` (
+  `id` int NOT NULL AUTO_INCREMENT,
+  `metric_key` varchar(190) NOT NULL COMMENT 'Metric key: `String` (limit 190 characters). Part of *Primary Key* for ``metrics`` table.',
+  `value` float NOT NULL COMMENT 'Metric value: `Float`. Defined as *Non-null* in schema.',
+  `worker_index` varchar(32) NOT NULL COMMENT 'Metric worker_index: `String` (limit 32 characters). Part of *Primary Key* for\r\n    ``metrics`` table.',
+  `timestamp` bigint(20) NOT NULL COMMENT 'Timestamp recorded for this metric entry: `BigInteger`. Part of *Primary Key* for   ``metrics`` table.',
+  `step` bigint(11) NOT NULL COMMENT 'Step recorded for this metric entry: `BigInteger`.',
+  `is_nan` int(11) NOT NULL COMMENT 'True if the value is in fact NaN.',
+  `job_name` varchar(32) NOT NULL COMMENT 'JOB NAME to which this metric belongs to: Part of *Primary Key* for ``metrics`` table.',
+  PRIMARY KEY  (`id`)
+) ENGINE=InnoDB DEFAULT CHARSET=utf8;
+
+-- ----------------------------
+-- Table structure for param
+-- ----------------------------
+DROP TABLE IF EXISTS `param`;
+CREATE TABLE `param` (

Review comment:
       ```suggestion
   DROP TABLE IF EXISTS `params`;
   CREATE TABLE `params` (
   ```
   pysubmarine tracking API will log parameters to the table named `params`. so here we need to change to `params`




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

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



[GitHub] [submarine] asfgit closed pull request #311: SUBMARINE-500. Add metric param instance to the submarine-server rest

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #311:
URL: https://github.com/apache/submarine/pull/311


   


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

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



[GitHub] [submarine] liuxunorg commented on a change in pull request #311: SUBMARINE-500. Add metric, param instance to the submarine-server rest

Posted by GitBox <gi...@apache.org>.
liuxunorg commented on a change in pull request #311:
URL: https://github.com/apache/submarine/pull/311#discussion_r443286533



##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/database/service/ParamService.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import java.util.List;
+
+import org.apache.ibatis.session.SqlSession;
+import org.apache.submarine.server.workbench.database.MyBatisUtil;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.mappers.ParamMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ParamService {
+  
+  private static final Logger LOG = LoggerFactory.getLogger(ParamService.class);
+
+  public List<Param> selectAll() throws Exception {
+    LOG.info("Param selectAll");
+    List<Param> params;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      params = mapper.selectAll();
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/database/service/MetricService.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import org.apache.submarine.server.workbench.database.MyBatisUtil;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.mappers.MetricMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import org.apache.ibatis.session.SqlSession;
+
+public class MetricService {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricService.class);
+
+  public MetricService() {
+  }
+
+  public List<Metric> selectAll() throws Exception {
+    List<Metric> result;
+    LOG.info("Metric selectAll");
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      result = mapper.selectAll();
+      sqlSession.commit();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return result;
+  }
+  
+  public boolean deleteById(String id) throws Exception {
+    LOG.info("Metric deleteByPrimaryKey {}", id);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      mapper.deleteById(id);
+      sqlSession.commit();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+
+  public boolean insert(Metric metric) throws Exception {
+    LOG.info("Metric insert {}", metric);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      mapper.insert(metric);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+
+  public Metric selectById(String id) throws Exception {
+    LOG.info("Metric selectByPrimaryKey {}", id);
+    Metric metric;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      metric = mapper.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return metric;
+  }
+  
+  public boolean update(Metric metric) throws Exception {
+    LOG.info("Metric update {}", metric);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      mapper.update(metric);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+
+  public List<Metric> selectByPrimaryKeySelective(Metric metric) throws Exception {
+    List<Metric> result;
+    LOG.info("Metric selectByPrimaryKeySelective");
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      result = mapper.selectByPrimaryKeySelective(metric);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.update(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/database/service/MetricService.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import org.apache.submarine.server.workbench.database.MyBatisUtil;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.mappers.MetricMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import org.apache.ibatis.session.SqlSession;
+
+public class MetricService {
+  private static final Logger LOG = LoggerFactory.getLogger(MetricService.class);
+
+  public MetricService() {
+  }
+
+  public List<Metric> selectAll() throws Exception {
+    List<Metric> result;
+    LOG.info("Metric selectAll");
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      result = mapper.selectAll();
+      sqlSession.commit();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return result;
+  }
+  
+  public boolean deleteById(String id) throws Exception {
+    LOG.info("Metric deleteByPrimaryKey {}", id);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      mapper.deleteById(id);
+      sqlSession.commit();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+
+  public boolean insert(Metric metric) throws Exception {
+    LOG.info("Metric insert {}", metric);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      mapper.insert(metric);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+
+  public Metric selectById(String id) throws Exception {
+    LOG.info("Metric selectByPrimaryKey {}", id);
+    Metric metric;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      MetricMapper mapper = sqlSession.getMapper(MetricMapper.class);
+      metric = mapper.selectById(id);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.update(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @POST
+  @Path("/selective")
+  @SubmarineApi
+  public Response selectByPrimaryKeySelective(Metric metric) {
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/database/service/ParamService.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import java.util.List;
+
+import org.apache.ibatis.session.SqlSession;
+import org.apache.submarine.server.workbench.database.MyBatisUtil;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.mappers.ParamMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ParamService {
+  
+  private static final Logger LOG = LoggerFactory.getLogger(ParamService.class);
+
+  public List<Param> selectAll() throws Exception {
+    LOG.info("Param selectAll");
+    List<Param> params;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      params = mapper.selectAll();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return params;
+  }
+  
+  public boolean deleteById(String id) throws Exception {
+    LOG.info("Param deleteById {}", id);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      mapper.deleteById(id);
+      sqlSession.commit();
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();

Review comment:
       ```
   return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
   ```
   1. Response.Status.OK: Indicates that the rest request was successful
   2. success(true) or success(false): Indicates whether the background processing is normal or abnormal
   3. result(false): Indicates that the background processing is normal, then the processing result is returned.
   If the background is abnormal, there is no need to return any value. The caller determines whether there is a return value based on the success value.
   
   SO,Please change 
   ```
   return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
   ```
   to
   ```
   return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(false).build();
   ```

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/database/service/ParamService.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import java.util.List;
+
+import org.apache.ibatis.session.SqlSession;
+import org.apache.submarine.server.workbench.database.MyBatisUtil;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.mappers.ParamMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ParamService {
+  
+  private static final Logger LOG = LoggerFactory.getLogger(ParamService.class);
+
+  public List<Param> selectAll() throws Exception {
+    LOG.info("Param selectAll");
+    List<Param> params;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      params = mapper.selectAll();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return params;
+  }
+  
+  public boolean deleteById(String id) throws Exception {
+    LOG.info("Param deleteById {}", id);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      mapper.deleteById(id);
+      sqlSession.commit();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+
+  public boolean insert(Param param) throws Exception {
+    LOG.info("Param insert {}", param);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      mapper.insert(param);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+  
+  public Param selectById(String id) throws Exception {
+    LOG.info("Param selectById {}", id);
+    Param param;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      param = mapper.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return param;
+  }
+
+  public boolean update(Param param) throws Exception {
+    LOG.info("Param update {}", param);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      mapper.update(param);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+
+  public List<Param> selectByPrimaryKeySelective(Param param) throws Exception {
+    List<Param> result;
+    LOG.info("Param selectByPrimaryKeySelective");
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      result = mapper.selectByPrimaryKeySelective(param);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/database/service/ParamService.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.submarine.server.workbench.database.service;
+
+import java.util.List;
+
+import org.apache.ibatis.session.SqlSession;
+import org.apache.submarine.server.workbench.database.MyBatisUtil;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.mappers.ParamMapper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class ParamService {
+  
+  private static final Logger LOG = LoggerFactory.getLogger(ParamService.class);
+
+  public List<Param> selectAll() throws Exception {
+    LOG.info("Param selectAll");
+    List<Param> params;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      params = mapper.selectAll();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return params;
+  }
+  
+  public boolean deleteById(String id) throws Exception {
+    LOG.info("Param deleteById {}", id);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      mapper.deleteById(id);
+      sqlSession.commit();
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+
+  public boolean insert(Param param) throws Exception {
+    LOG.info("Param insert {}", param);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      mapper.insert(param);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new Exception(e);
+    }
+    return true;
+  }
+  
+  public Param selectById(String id) throws Exception {
+    LOG.info("Param selectById {}", id);
+    Param param;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ParamMapper mapper = sqlSession.getMapper(ParamMapper.class);
+      param = mapper.selectById(id);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteParam(@QueryParam("id") String id) {
+    LOG.info("deleteParam ({})", id);
+    boolean result = false;
+    try {
+      result = paramService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteParam(@QueryParam("id") String id) {
+    LOG.info("deleteParam ({})", id);
+    boolean result = false;
+    try {
+      result = paramService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putParam(Param param) {
+    LOG.info("putParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.update(param);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/MetricRestApi.java
##########
@@ -0,0 +1,181 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Metric;
+import org.apache.submarine.server.workbench.database.service.MetricService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/metric")
+@Produces("application/json")
+@Singleton
+public class MetricRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  MetricService metricService = new MetricService();
+
+  @Inject
+  public MetricRestApi() {
+  }
+
+  /*
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | key   | value    | worker_index | timestamp     | step | is_nan | job_name         |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  # | score | 0.666667 | worker-1     | 1569139525097 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569149139731 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569169376482 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236290721 |    0 |      0 | application_1234 |
+  # | score | 0.666667 | worker-1     | 1569236466722 |    0 |      0 | application_1234 |
+  # +-------+----------+--------------+---------------+------+--------+------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listMetric(@QueryParam("metricKey") String metricKey,
+                              @QueryParam("value") Float value,
+                              @QueryParam("workerIndex") String workerIndex,
+                              @QueryParam("timestamp") BigInteger timestamp,
+                              @QueryParam("step") Integer step,
+                              @QueryParam("isNan") Integer isNan,
+                              @QueryParam("jobName") String jobName,
+                              @QueryParam("id") String id) {
+
+    Metric metric = new Metric();
+    metric.setMetricKey(metricKey);
+    metric.setValue(value);
+    metric.setWorkerIndex(workerIndex);
+    metric.setTimestamp(timestamp);
+    metric.setStep(step);
+    metric.setIsNan(isNan);
+    metric.setJobName(jobName);
+    metric.setId(id);
+
+    LOG.info("listMetric ({})", metric);
+
+    List<Metric> metrics;
+    try {
+      metrics = metricService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Metric>>(Response.Status.OK).success(true).result(metrics).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getMetric(@PathParam("id") String id) {
+    Metric metric;
+    try {
+      metric = metricService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Metric>(Response.Status.OK).success(true).result(metric).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.insert(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteMetric(@QueryParam("id") String id) {
+    boolean result = false;
+    try {
+      result = metricService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putMetric(Metric metric) {
+    boolean result = false;
+    try {
+      result = metricService.update(metric);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteParam(@QueryParam("id") String id) {
+    LOG.info("deleteParam ({})", id);
+    boolean result = false;
+    try {
+      result = paramService.deleteById(id);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteParam(@QueryParam("id") String id) {
+    LOG.info("deleteParam ({})", id);
+    boolean result = false;
+    try {
+      result = paramService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putParam(Param param) {
+    LOG.info("putParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.update(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteParam(@QueryParam("id") String id) {
+    LOG.info("deleteParam ({})", id);
+    boolean result = false;
+    try {
+      result = paramService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putParam(Param param) {
+    LOG.info("putParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.update(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @POST
+  @Path("/selective")
+  @SubmarineApi
+  public Response selectByPrimaryKeySelective(Param metric) {
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(metric);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/java/org/apache/submarine/server/workbench/rest/ParamRestApi.java
##########
@@ -0,0 +1,177 @@
+/*
+ * 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.submarine.server.workbench.rest;
+
+import org.apache.submarine.server.workbench.annotation.SubmarineApi;
+import org.apache.submarine.server.workbench.database.entity.Param;
+import org.apache.submarine.server.workbench.database.service.ParamService;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+import javax.inject.Inject;
+import javax.inject.Singleton;
+import javax.ws.rs.DELETE;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.PUT;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.Response;
+
+@Path("/param")
+@Produces("application/json")
+@Singleton
+public class ParamRestApi {
+  private static final Logger LOG = LoggerFactory.getLogger(LoginRestApi.class);
+  ParamService paramService = new ParamService();
+
+  @Inject
+  public ParamRestApi() {
+  }
+
+  /*
+  # +----------+-------+--------------+-----------------------+
+  # | key      | value | worker_index | job_name              |
+  # +----------+-------+--------------+-----------------------+
+  # | max_iter | 100   | worker-1     | application_123651651 |
+  # | n_jobs   | 5     | worker-1     | application_123456898 |
+  # | alpha    | 20    | worker-1     | application_123456789 |
+  # +----------+-------+--------------+-----------------------+
+  */
+
+  @GET
+  @Path("/list")
+  @SubmarineApi
+  public Response listParam(@QueryParam("id") String id, 
+                            @QueryParam("paramKey") String paramKey, 
+                            @QueryParam("value") String value, 
+                            @QueryParam("workerIndex") String workerIndex, 
+                            @QueryParam("jobName") String jobName) {
+    
+    Param param = new Param();
+    param.setId(id);
+    param.setParamKey(paramKey);
+    param.setValue(value);
+    param.setWorkerIndex(workerIndex);
+    param.setJobName(jobName);
+
+    LOG.info("listParam ({})", param);
+    
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+      
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<List<Param>>(Response.Status.OK).success(true).result(params).build();
+  }
+
+  @GET
+  @Path("/{id}")
+  @SubmarineApi
+  public Response getParam(@PathParam("id") String id) {
+    LOG.info("getParam ({})", id);
+    
+    Param param;
+    try {
+      param = paramService.selectById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Param>(Response.Status.OK).success(true).result(param).build();
+  }
+
+  @POST
+  @Path("/add")
+  @SubmarineApi
+  public Response postParam(Param param) {
+    LOG.info("postParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.insert(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @DELETE
+  @Path("/delete")
+  @SubmarineApi
+  public Response deleteParam(@QueryParam("id") String id) {
+    LOG.info("deleteParam ({})", id);
+    boolean result = false;
+    try {
+      result = paramService.deleteById(id);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @PUT
+  @Path("/edit")
+  @SubmarineApi
+  public Response putParam(Param param) {
+    LOG.info("putParam ({})", param);
+    boolean result = false;
+    try {
+      result = paramService.update(param);
+
+    } catch (Exception e) {
+      LOG.error(e.toString());
+
+      return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(false).build();
+    }
+    return new JsonResponse.Builder<Boolean>(Response.Status.OK).success(true).result(result).build();
+  }
+
+  @POST
+  @Path("/selective")
+  @SubmarineApi
+  public Response selectByPrimaryKeySelective(Param metric) {
+    List<Param> params;
+    try {
+      params = paramService.selectByPrimaryKeySelective(metric);
+

Review comment:
       Please delete this a blank line.

##########
File path: submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/MetricMapper.xml
##########
@@ -0,0 +1,105 @@
+<?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.
+-->
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd">
+<mapper namespace="org.apache.submarine.server.workbench.database.mappers.MetricMapper">
+  <resultMap id="BaseEntityResultMap" type="org.apache.submarine.server.workbench.database.entity.BaseEntity">
+    <id property="id" column="id"/>
+  </resultMap>
+
+  <resultMap id="resultMap" type="org.apache.submarine.server.workbench.database.entity.Metric" extends="BaseEntityResultMap">
+    <result column="metric_key" jdbcType="VARCHAR" property="metricKey" />
+    <result column="value" jdbcType="FLOAT" property="value" />
+    <result column="worker_index" jdbcType="VARCHAR" property="workerIndex" />
+    <result column="timestamp" jdbcType="BIGINT" property="timestamp" />
+    <result column="step" jdbcType="INTEGER" property="step" />
+    <result column="is_nan" jdbcType="INTEGER" property="isNan" />
+    <result column="job_name" jdbcType="VARCHAR" property="jobName" />
+  </resultMap>
+
+  <sql id="Base_Column_List">
+    id, metric_key, value, worker_index, timestamp, step, is_nan, job_name
+  </sql>
+
+  <select id="selectAll" parameterType="java.lang.String" resultMap="resultMap">
+    select
+    <include refid="Base_Column_List" />
+    from metric
+    where 1 = 1
+  </select>
+
+  <select id="selectById" parameterType="java.lang.String" resultMap="resultMap">
+    select
+    <include refid="Base_Column_List" />
+    from metric
+    where id = #{id,jdbcType=VARCHAR}
+  </select>
+
+  <delete id="deleteById" parameterType="java.lang.String">
+    delete from metric
+    where id = #{id,jdbcType=VARCHAR}
+  </delete>
+
+  <insert id="insert" parameterType="org.apache.submarine.server.workbench.database.entity.Metric"
+          useGeneratedKeys="true" keyProperty="id">
+    insert into metric (metric_key, value, worker_index, timestamp,
+      step, is_nan, job_name)
+    values (#{metricKey,jdbcType=VARCHAR}, 
+      #{value,jdbcType=FLOAT},
+      #{workerIndex,jdbcType=VARCHAR}, 
+      #{timestamp,jdbcType=BIGINT}, 
+      #{step,jdbcType=INTEGER},
+      #{isNan,jdbcType=INTEGER}, 
+      #{jobName,jdbcType=VARCHAR})
+  </insert>
+  
+  <update id="update" parameterType="org.apache.submarine.server.workbench.database.entity.Metric">
+    update metric
+    set metric_key = #{metricKey,jdbcType=VARCHAR},
+      value = #{value,jdbcType=FLOAT},
+      worker_index = #{workerIndex,jdbcType=VARCHAR},
+      timestamp = #{timestamp,jdbcType=BIGINT},
+      step = #{step,jdbcType=INTEGER},
+      is_nan = #{isNan,jdbcType=INTEGER},
+      job_name = #{jobName,jdbcType=VARCHAR}
+    where id = #{id,jdbcType=VARCHAR}
+  </update>
+  
+  <select id="selectByPrimaryKeySelective" parameterType="java.lang.String" resultMap="resultMap">
+  select
+  <include refid="Base_Column_List" />
+  from metric
+  where 1 = 1
+    <if test="metricKey != null">
+      AND metric_key = #{metricKey,jdbcType=VARCHAR}
+    </if>
+    <if test="timestamp != null">
+      AND timestamp = #{timestamp,jdbcType=BIGINT}
+    </if>
+    <if test="workerIndex != null">
+      AND worker_index = #{workerIndex,jdbcType=INTEGER}
+    </if>
+    <if test="jobName != null">
+      AND job_name = #{jobName,jdbcType=VARCHAR}
+    </if>
+    <if test="id != null">
+      AND id = #{id,jdbcType=VARCHAR}
+    </if>

Review comment:
       Need Indent 2 spaces.




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

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



[GitHub] [submarine] JohnTing commented on pull request #311: SUBMARINE-500. Add metric param instance to the submarine-server rest

Posted by GitBox <gi...@apache.org>.
JohnTing commented on pull request #311:
URL: https://github.com/apache/submarine/pull/311#issuecomment-650586280


   > hi @JohnTing , Have you completed all the changes?
   
   Yes, all the changes are completed.


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

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



[GitHub] [submarine] liuxunorg commented on pull request #311: SUBMARINE-500. Add metric, param instance to the submarine-server rest

Posted by GitBox <gi...@apache.org>.
liuxunorg commented on pull request #311:
URL: https://github.com/apache/submarine/pull/311#issuecomment-650574146


   hi @JohnTing , Have you completed all the changes?


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

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