You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@submarine.apache.org by pi...@apache.org on 2021/01/30 10:58:44 UTC

[submarine] branch master updated: SUBMARINE-714. Store experiment metadata into mysql database

This is an automated email from the ASF dual-hosted git repository.

pingsutw pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/submarine.git


The following commit(s) were added to refs/heads/master by this push:
     new e28acdc  SUBMARINE-714. Store experiment metadata into mysql database
e28acdc is described below

commit e28acdc03a9e996da2095bc27c6d1cfca45eec72
Author: Byron <by...@gmail.com>
AuthorDate: Fri Jan 22 18:29:33 2021 +0800

    SUBMARINE-714. Store experiment metadata into mysql database
    
    ### What is this PR for?
    In the past, we temporarily stored experiment metadata in memory. However, it is not persistent, when we restart the server, the data will be lost. Alternatively, we should store it in MySQL database.
    
    ### What type of PR is it?
    [Improvement]
    
    ### Todos
    * [ ] - Task
    
    ### What is the Jira issue?
    https://issues.apache.org/jira/browse/SUBMARINE-714
    
    ### How should this be tested?
    
    ### Screenshots (if appropriate)
    
    ### Questions:
    * Does the licenses files need update? No
    * Is there breaking changes for older versions? No
    * Does this needs documentation? No
    
    Author: Byron <by...@gmail.com>
    
    Signed-off-by: Kevin <pi...@apache.org>
    
    Closes #496 from ByronHsu/EXPERIMENT_DB and squashes the following commits:
    
    3d0e5edc [Byron] add license
    c42e209f [Byron] rebuild experiment if needed and remove cacheMap
    6b530fd4 [Byron] remove unused function
    722411e9 [Byron] revert editor config
    c8e7d850 [Byron] add license
    890e3699 [Byron] use experiment service in manager
    ea093bea [Byron] experiment service test succeeds
---
 docs/database/submarine.sql                        |  14 ++
 .../submarine/templates/submarine-database.yaml    |   4 +
 .../submarine/templates/submarine-server.yaml      |   2 +
 helm-charts/submarine/values.yaml                  |   2 +
 .../server/experiment/ExperimentManager.java       | 180 +++++++++-----
 .../experiment/database/ExperimentEntity.java      |  51 ++++
 .../experiment/database/ExperimentMapper.java      |  31 +++
 .../experiment/database/ExperimentService.java     | 103 ++++++++
 .../src/main/resources/mybatis-config.xml          |   1 +
 .../database/mappers/ExperimentMapper.xml          |  73 ++++++
 .../server/experiment/ExperimentManagerTest.java   | 263 +++++++++++++++++++++
 .../experiment/database/ExperimentServiceTest.java | 119 ++++++++++
 .../src/test/resources/experiment/new_spec.json    |  20 ++
 .../src/test/resources/experiment/result.json      |   7 +
 .../src/test/resources/experiment/spec.json        |  20 ++
 .../src/test/resources/experiment/status.json      |   5 +
 16 files changed, 840 insertions(+), 55 deletions(-)

diff --git a/docs/database/submarine.sql b/docs/database/submarine.sql
index 76dfc49..56d51a5 100644
--- a/docs/database/submarine.sql
+++ b/docs/database/submarine.sql
@@ -237,6 +237,20 @@ CREATE TABLE `environment` (
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8;
 
 -- ----------------------------
+-- Table structure for experiment
+-- ----------------------------
+DROP TABLE IF EXISTS `experiment`;
+CREATE TABLE `experiment` (
+  `id` varchar(64) NOT NULL COMMENT 'Id of the Experiment',
+  `experiment_spec` text NOT NULL COMMENT 'Spec of the experiment',
+  `create_by` varchar(32) DEFAULT NULL COMMENT 'create user',
+  `create_time` datetime DEFAULT NULL COMMENT 'create time',
+  `update_by` varchar(32) DEFAULT NULL COMMENT 'last update user',
+  `update_time` datetime DEFAULT NULL COMMENT 'last update time',
+   PRIMARY KEY `id` (`id`)
+) ENGINE=InnoDB DEFAULT CHARSET=utf8;
+
+-- ----------------------------
 -- Table structure for metric
 -- ----------------------------
 DROP TABLE IF EXISTS `metrics`;
diff --git a/helm-charts/submarine/templates/submarine-database.yaml b/helm-charts/submarine/templates/submarine-database.yaml
index 02803a8..bd0ab96 100644
--- a/helm-charts/submarine/templates/submarine-database.yaml
+++ b/helm-charts/submarine/templates/submarine-database.yaml
@@ -28,6 +28,10 @@ spec:
     metadata:
       labels:
         app: "{{ .Values.submarine.database.name }}"
+      {{ if .Values.submarine.database.dev }}
+      annotations:
+        timestamp: {{ now | quote }}
+      {{ end }}
     spec:
       containers:
         - name: "{{ .Values.submarine.database.name }}"
diff --git a/helm-charts/submarine/templates/submarine-server.yaml b/helm-charts/submarine/templates/submarine-server.yaml
index b27f62f..2452115 100644
--- a/helm-charts/submarine/templates/submarine-server.yaml
+++ b/helm-charts/submarine/templates/submarine-server.yaml
@@ -48,8 +48,10 @@ spec:
     metadata:
       labels:
         run: "{{ .Values.submarine.server.name }}"
+      {{ if .Values.submarine.server.dev }}
       annotations:
         timestamp: {{ now | quote }}
+      {{ end }}
     spec:
       serviceAccountName: "{{ .Values.submarine.server.name }}"
       containers:
diff --git a/helm-charts/submarine/values.yaml b/helm-charts/submarine/values.yaml
index 2571ddd..23f668d 100644
--- a/helm-charts/submarine/values.yaml
+++ b/helm-charts/submarine/values.yaml
@@ -22,6 +22,7 @@ submarine:
     name: submarine-server
     image: apache/submarine:server-0.6.0-SNAPSHOT
     servicePort: 8080
+    dev: false # if true, restart server pod every time at helm upgrade
   database:
     imagePullPolicy: IfNotPresent
     replicas: 1
@@ -29,6 +30,7 @@ submarine:
     image: apache/submarine:database-0.6.0-SNAPSHOT
     servicePort: 3306
     mysqlRootPassword: password
+    dev: false # if true, restart database pod every time at helm upgrade
   traefik:
     enabled: true
   tensorboard:
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/ExperimentManager.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/ExperimentManager.java
index 92c8da9..039b17e 100644
--- a/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/ExperimentManager.java
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/ExperimentManager.java
@@ -21,13 +21,15 @@ package org.apache.submarine.server.experiment;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import javax.ws.rs.core.Response.Status;
 
+import com.google.common.annotations.VisibleForTesting;
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
 import org.apache.submarine.commons.utils.SubmarineConfiguration;
 import org.apache.submarine.commons.utils.exception.SubmarineRuntimeException;
 import org.apache.submarine.server.SubmarineServer;
@@ -38,6 +40,8 @@ import org.apache.submarine.server.api.Submitter;
 import org.apache.submarine.server.api.experiment.ExperimentLog;
 import org.apache.submarine.server.api.experiment.TensorboardInfo;
 import org.apache.submarine.server.api.spec.ExperimentSpec;
+import org.apache.submarine.server.experiment.database.ExperimentEntity;
+import org.apache.submarine.server.experiment.database.ExperimentService;
 import org.apache.submarine.server.rest.RestConstants;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -60,6 +64,7 @@ public class ExperimentManager {
   private final ConcurrentMap<String, Experiment> cachedExperimentMap = new ConcurrentHashMap<>();
 
   private final Submitter submitter;
+  private final ExperimentService experimentService;
 
   /**
    * Get the singleton instance
@@ -70,15 +75,17 @@ public class ExperimentManager {
     if (manager == null) {
       synchronized (ExperimentManager.class) {
         if (manager == null) {
-          manager = new ExperimentManager(SubmitterManager.loadSubmitter());
+          manager = new ExperimentManager(SubmitterManager.loadSubmitter(), new ExperimentService());
         }
       }
     }
     return manager;
   }
 
-  private ExperimentManager(Submitter submitter) {
+  @VisibleForTesting
+  protected ExperimentManager(Submitter submitter, ExperimentService experimentService) {
     this.submitter = submitter;
+    this.experimentService = experimentService;
   }
 
   /**
@@ -88,6 +95,7 @@ public class ExperimentManager {
    * @return object
    * @throws SubmarineRuntimeException the service error
    */
+
   public Experiment createExperiment(ExperimentSpec spec) throws SubmarineRuntimeException {
     checkSpec(spec);
 
@@ -106,24 +114,14 @@ public class ExperimentManager {
     spec.getMeta().getEnvVars().remove(RestConstants.JOB_ID);
     spec.getMeta().getEnvVars().remove(RestConstants.SUBMARINE_TRACKING_URI);
     experiment.setSpec(spec);
-    cachedExperimentMap.putIfAbsent(experiment.getExperimentId().toString(), experiment);
-    return experiment;
-  }
 
-  private String getSQLAlchemyURL() {
-    SubmarineConfiguration conf = SubmarineConfiguration.getInstance();
-    String jdbcUrl = conf.getJdbcUrl();
-    jdbcUrl = jdbcUrl.substring(jdbcUrl.indexOf("//") + 2, jdbcUrl.indexOf("?"));
-    String jdbcUserName = conf.getJdbcUserName();
-    String jdbcPassword = conf.getJdbcPassword();
-    return "mysql+pymysql://" + jdbcUserName + ":" + jdbcPassword + "@" + jdbcUrl;
-  }
+    ExperimentEntity entity = buildEntityFromExperiment(experiment);
+    experimentService.insert(entity);
 
-  private ExperimentId generateExperimentId() {
-    return ExperimentId.newInstance(SubmarineServer.getServerTimeStamp(),
-        experimentCounter.incrementAndGet());
+    return experiment;
   }
 
+
   /**
    * Get experiment
    *
@@ -133,10 +131,12 @@ public class ExperimentManager {
    */
   public Experiment getExperiment(String id) throws SubmarineRuntimeException {
     checkExperimentId(id);
-    Experiment experiment = cachedExperimentMap.get(id);
-    ExperimentSpec spec = experiment.getSpec();
-    Experiment patchExperiment = submitter.findExperiment(spec);
-    experiment.rebuild(patchExperiment);
+
+    ExperimentEntity entity = experimentService.select(id);
+    Experiment experiment = buildExperimentFromEntity(entity);
+    Experiment foundExperiment = submitter.findExperiment(experiment.getSpec());
+    experiment.rebuild(foundExperiment);
+
     return experiment;
   }
 
@@ -149,13 +149,15 @@ public class ExperimentManager {
    */
   public List<Experiment> listExperimentsByStatus(String status) throws SubmarineRuntimeException {
     List<Experiment> experimentList = new ArrayList<>();
-    for (Map.Entry<String, Experiment> entry : cachedExperimentMap.entrySet()) {
-      Experiment experiment = entry.getValue();
-      ExperimentSpec spec = experiment.getSpec();
-      Experiment patchExperiment = submitter.findExperiment(spec);
-      LOG.info("Found experiment: {}", patchExperiment.getStatus());
-      if (status == null || status.toLowerCase().equals(patchExperiment.getStatus().toLowerCase())) {
-        experiment.rebuild(patchExperiment);
+    List<ExperimentEntity> entities = experimentService.selectAll();
+
+    for (ExperimentEntity entity: entities) {
+      Experiment experiment = buildExperimentFromEntity(entity);
+      Experiment foundExperiment = submitter.findExperiment(experiment.getSpec());
+
+      LOG.info("Found experiment: {}", foundExperiment.getStatus());
+      if (status == null || status.toLowerCase().equals(foundExperiment.getStatus().toLowerCase())) {
+        experiment.rebuild(foundExperiment);
         experimentList.add(experiment);
       }
     }
@@ -167,17 +169,28 @@ public class ExperimentManager {
    * Patch the experiment
    *
    * @param id   experiment id
-   * @param spec spec
+   * @param newSpec spec
    * @return object
    * @throws SubmarineRuntimeException the service error
    */
-  public Experiment patchExperiment(String id, ExperimentSpec spec) throws SubmarineRuntimeException {
+  public Experiment patchExperiment(String id, ExperimentSpec newSpec) throws SubmarineRuntimeException {
     checkExperimentId(id);
-    checkSpec(spec);
-    Experiment experiment = cachedExperimentMap.get(id);
-    Experiment patchExperiment = submitter.patchExperiment(spec);
-    experiment.setSpec(spec);
+    checkSpec(newSpec);
+
+    ExperimentEntity entity = experimentService.select(id);
+    Experiment experiment = buildExperimentFromEntity(entity);
+    Experiment patchExperiment = submitter.patchExperiment(newSpec);
+
+    // update spec in returned experiment
+    experiment.setSpec(newSpec);
+
+    // update entity and commit
+    entity.setExperimentSpec(new GsonBuilder().disableHtmlEscaping().create().toJson(newSpec));
+    experimentService.update(entity);
+
+    // patch new information in experiment
     experiment.rebuild(patchExperiment);
+
     return experiment;
   }
 
@@ -190,10 +203,13 @@ public class ExperimentManager {
    */
   public Experiment deleteExperiment(String id) throws SubmarineRuntimeException {
     checkExperimentId(id);
-    Experiment experiment = cachedExperimentMap.remove(id);
-    ExperimentSpec spec = experiment.getSpec();
-    Experiment patchExperiment = submitter.deleteExperiment(spec);
-    experiment.rebuild(patchExperiment);
+
+    ExperimentEntity entity = experimentService.select(id);
+    Experiment experiment = buildExperimentFromEntity(entity);
+    Experiment deletedExperiment = submitter.deleteExperiment(experiment.getSpec());
+    experimentService.delete(id);
+
+    experiment.rebuild(deletedExperiment);
     return experiment;
   }
 
@@ -205,17 +221,24 @@ public class ExperimentManager {
    * @throws SubmarineRuntimeException the service error
    */
   public List<ExperimentLog> listExperimentLogsByStatus(String status) throws SubmarineRuntimeException {
-    List<ExperimentLog> experimentLogList = new ArrayList<ExperimentLog>();
-    for (Map.Entry<String, Experiment> entry : cachedExperimentMap.entrySet()) {
-      String id = entry.getKey();
-      Experiment experiment = entry.getValue();
-      ExperimentSpec spec = experiment.getSpec();
-      Experiment patchExperiment = submitter.findExperiment(spec);
-      LOG.info("Found experiment: {}", patchExperiment.getStatus());
-      if (status == null || status.toLowerCase().equals(patchExperiment.getStatus().toLowerCase())) {
-        experiment.rebuild(patchExperiment);
-        experimentLogList.add(submitter.getExperimentLogName(spec, id));
+    List<ExperimentLog> experimentLogList = new ArrayList<>();
+    List<ExperimentEntity> entities = experimentService.selectAll();
+
+    for (ExperimentEntity entity: entities) {
+      Experiment experiment = buildExperimentFromEntity(entity);
+      Experiment foundExperiment = submitter.findExperiment(experiment.getSpec());
+
+      LOG.info("Found experiment: {}", foundExperiment.getStatus());
+
+      if (status == null || status.toLowerCase().equals(foundExperiment.getStatus().toLowerCase())) {
+        experiment.rebuild(foundExperiment);
+
+        experimentLogList.add(submitter.getExperimentLogName(
+            experiment.getSpec(),
+            experiment.getExperimentId().toString()
+        ));
       }
+
     }
     return experimentLogList;
   }
@@ -229,11 +252,17 @@ public class ExperimentManager {
    */
   public ExperimentLog getExperimentLog(String id) throws SubmarineRuntimeException {
     checkExperimentId(id);
-    Experiment experiment = cachedExperimentMap.get(id);
-    ExperimentSpec spec = experiment.getSpec();
-    Experiment patchExperiment = submitter.findExperiment(spec);
-    experiment.rebuild(patchExperiment);
-    return submitter.getExperimentLog(spec, id);
+
+    ExperimentEntity entity = experimentService.select(id);
+    Experiment experiment = buildExperimentFromEntity(entity);
+
+    Experiment foundExperiment = submitter.findExperiment(experiment.getSpec());
+    experiment.rebuild(foundExperiment);
+
+    return submitter.getExperimentLogName(
+      experiment.getSpec(),
+      experiment.getExperimentId().toString()
+    );
   }
 
   /**
@@ -253,9 +282,50 @@ public class ExperimentManager {
   }
 
   private void checkExperimentId(String id) throws SubmarineRuntimeException {
-    ExperimentId experimentId = ExperimentId.fromString(id);
-    if (experimentId == null || !cachedExperimentMap.containsKey(id)) {
+    ExperimentEntity entity = experimentService.select(id);
+    if (entity == null) {
       throw new SubmarineRuntimeException(Status.NOT_FOUND.getStatusCode(), "Not found experiment.");
     }
   }
+
+  private String getSQLAlchemyURL() {
+    SubmarineConfiguration conf = SubmarineConfiguration.getInstance();
+    String jdbcUrl = conf.getJdbcUrl();
+    jdbcUrl = jdbcUrl.substring(jdbcUrl.indexOf("//") + 2, jdbcUrl.indexOf("?"));
+    String jdbcUserName = conf.getJdbcUserName();
+    String jdbcPassword = conf.getJdbcPassword();
+    return "mysql+pymysql://" + jdbcUserName + ":" + jdbcPassword + "@" + jdbcUrl;
+  }
+
+  public ExperimentId generateExperimentId() {
+    return ExperimentId.newInstance(SubmarineServer.getServerTimeStamp(),
+      experimentCounter.incrementAndGet());
+  }
+
+  /**
+   * Create a new experiment instance from entity, and filled
+   *   1. experimentId
+   *   2. spec
+   * @param entity
+   * @return Experiment
+   */
+  private Experiment buildExperimentFromEntity(ExperimentEntity entity) {
+    Experiment experiment = new Experiment();
+    experiment.setExperimentId(ExperimentId.fromString(entity.getId()));
+    experiment.setSpec(new Gson().fromJson(entity.getExperimentSpec(), ExperimentSpec.class));
+    return experiment;
+  }
+
+  /**
+   * Create a ExperimentEntity instance from experiment
+   *
+   * @param experiment
+   * @return ExperimentEntity
+   */
+  private ExperimentEntity buildEntityFromExperiment(Experiment experiment) {
+    ExperimentEntity entity = new ExperimentEntity();
+    entity.setId(experiment.getExperimentId().toString());
+    entity.setExperimentSpec(new GsonBuilder().disableHtmlEscaping().create().toJson(experiment.getSpec()));
+    return entity;
+  }
 }
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentEntity.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentEntity.java
new file mode 100644
index 0000000..ef83de0
--- /dev/null
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentEntity.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.submarine.server.experiment.database;
+
+import org.apache.submarine.server.database.entity.BaseEntity;
+
+public class ExperimentEntity extends BaseEntity {
+  /*
+    Take id (inherited from BaseEntity) as the primary key for experiment table
+  */
+  private String experimentSpec;
+
+  public ExperimentEntity() {}
+
+  public String getExperimentSpec() {
+    return experimentSpec;
+  }
+
+  public void setExperimentSpec(String experimentSpec) {
+    this.experimentSpec = experimentSpec;
+  }
+
+  @Override
+  public String toString() {
+    return "ExperimentEntity{" +
+      "experimentSpec='" + experimentSpec + '\'' +
+      ", id='" + id + '\'' +
+      ", createBy='" + createBy + '\'' +
+      ", createTime=" + createTime +
+      ", updateBy='" + updateBy + '\'' +
+      ", updateTime=" + updateTime +
+      '}';
+  }
+}
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentMapper.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentMapper.java
new file mode 100644
index 0000000..386a48e
--- /dev/null
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentMapper.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.submarine.server.experiment.database;
+
+import java.util.List;
+
+public interface ExperimentMapper {
+  List<ExperimentEntity> selectAll();
+  ExperimentEntity select(String id);
+
+  int insert(ExperimentEntity experiment);
+  int update(ExperimentEntity experiment);
+  int delete(String id);
+}
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentService.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentService.java
new file mode 100644
index 0000000..c23bbfa
--- /dev/null
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/ExperimentService.java
@@ -0,0 +1,103 @@
+/*
+ * 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.experiment.database;
+
+import org.apache.ibatis.session.SqlSession;
+import org.apache.submarine.commons.utils.exception.SubmarineRuntimeException;
+import org.apache.submarine.server.database.utils.MyBatisUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+public class ExperimentService {
+
+  private static final Logger LOG = LoggerFactory.getLogger(ExperimentService.class);
+
+  public List<ExperimentEntity> selectAll() throws SubmarineRuntimeException {
+    LOG.info("Experiment selectAll");
+    List<ExperimentEntity> entities;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ExperimentMapper mapper = sqlSession.getMapper(ExperimentMapper.class);
+      entities = mapper.selectAll();
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new SubmarineRuntimeException("Unable to get experiment entities from database");
+    }
+    return entities;
+  }
+
+  public ExperimentEntity select(String id) throws SubmarineRuntimeException {
+    LOG.info("Experiment select " + id);
+    ExperimentEntity entity;
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ExperimentMapper mapper = sqlSession.getMapper(ExperimentMapper.class);
+      entity = mapper.select(id);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new SubmarineRuntimeException("Unable to get experiment entity from database");
+    }
+    return entity;
+  }
+
+  public boolean insert(ExperimentEntity experiment) throws SubmarineRuntimeException {
+    LOG.info("Experiment insert");
+    LOG.debug(experiment.toString());
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ExperimentMapper mapper = sqlSession.getMapper(ExperimentMapper.class);
+      mapper.insert(experiment);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new SubmarineRuntimeException("Unable to insert experiment entity to database");
+    }
+    return true;
+  }
+
+  public boolean update(ExperimentEntity experiment) throws SubmarineRuntimeException {
+    LOG.info("Experiment update");
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ExperimentMapper mapper = sqlSession.getMapper(ExperimentMapper.class);
+      mapper.update(experiment);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new SubmarineRuntimeException("Unable to update experiment entity in database");
+    }
+    return true;
+  }
+
+  public boolean delete(String id) throws SubmarineRuntimeException {
+    LOG.info("Experiment delete " + id);
+
+    try (SqlSession sqlSession = MyBatisUtil.getSqlSession()) {
+      ExperimentMapper mapper = sqlSession.getMapper(ExperimentMapper.class);
+      mapper.delete(id);
+      sqlSession.commit();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new SubmarineRuntimeException("Unable to delete experiment entity from database");
+    }
+    return true;
+  }
+}
diff --git a/submarine-server/server-core/src/main/resources/mybatis-config.xml b/submarine-server/server-core/src/main/resources/mybatis-config.xml
index 06e4991..2982281 100755
--- a/submarine-server/server-core/src/main/resources/mybatis-config.xml
+++ b/submarine-server/server-core/src/main/resources/mybatis-config.xml
@@ -68,5 +68,6 @@
     <mapper resource='org/apache/submarine/database/mappers/ParamMapper.xml'/>
     <mapper resource='org/apache/submarine/database/mappers/EnvironmentMapper.xml'/>
     <mapper resource='org/apache/submarine/database/mappers/ExperimentTemplateMapper.xml'/>
+    <mapper resource='org/apache/submarine/database/mappers/ExperimentMapper.xml'/>
   </mappers>
 </configuration>
diff --git a/submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/ExperimentMapper.xml b/submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/ExperimentMapper.xml
new file mode 100644
index 0000000..4842da2
--- /dev/null
+++ b/submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/ExperimentMapper.xml
@@ -0,0 +1,73 @@
+<?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.experiment.database.ExperimentMapper">
+  <resultMap id="BaseEntityResultMap" type="org.apache.submarine.server.database.entity.BaseEntity">
+    <id property="id" column="id"/>
+    <result column="create_by" property="createBy"/>
+    <result column="create_time" property="createTime"/>
+    <result column="update_by" property="updateBy"/>
+    <result column="update_time" property="updateTime"/>
+  </resultMap>
+
+  <resultMap id="ExperimentEntityResultMap" type="org.apache.submarine.server.experiment.database.ExperimentEntity" extends="BaseEntityResultMap">
+    <result column="experiment_spec" jdbcType="VARCHAR" property="experimentSpec" />
+  </resultMap>
+
+  <sql id="Base_Column_List">
+    id, experiment_spec, create_by, create_time, update_by, update_time
+  </sql>
+
+  <select id="selectAll" parameterType="java.lang.String" resultMap="ExperimentEntityResultMap">
+    select
+    <include refid="Base_Column_List" />
+    from experiment
+  </select>
+
+  <select id="select" parameterType="java.lang.String" resultMap="ExperimentEntityResultMap">
+    select
+    <include refid="Base_Column_List" />
+    from experiment
+    where id = #{id,jdbcType=VARCHAR}
+  </select>
+
+  <delete id="delete" parameterType="java.lang.String">
+    delete from experiment
+    where id = #{id,jdbcType=VARCHAR}
+  </delete>
+
+  <insert id="insert" parameterType="org.apache.submarine.server.experiment.database.ExperimentEntity">
+    insert into experiment (id, experiment_spec, create_by, create_time, update_by, update_time)
+    values (#{id,jdbcType=VARCHAR}, #{experimentSpec,jdbcType=VARCHAR},
+            #{createBy,jdbcType=VARCHAR}, now(), #{updateBy,jdbcType=VARCHAR}, now())
+  </insert>
+
+  <update id="update" parameterType="org.apache.submarine.server.experiment.database.ExperimentEntity">
+    update experiment
+    <set>
+      <if test="experimentSpec != null">
+        experiment_spec = #{experimentSpec,jdbcType=VARCHAR},
+      </if>
+      update_time = now()
+    </set>
+    where id = #{id,jdbcType=VARCHAR}
+  </update>
+
+</mapper>
diff --git a/submarine-server/server-core/src/test/java/org/apache/submarine/server/experiment/ExperimentManagerTest.java b/submarine-server/server-core/src/test/java/org/apache/submarine/server/experiment/ExperimentManagerTest.java
new file mode 100644
index 0000000..578f86d
--- /dev/null
+++ b/submarine-server/server-core/src/test/java/org/apache/submarine/server/experiment/ExperimentManagerTest.java
@@ -0,0 +1,263 @@
+/*
+ * 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.experiment;
+
+
+import com.google.gson.Gson;
+import com.google.gson.GsonBuilder;
+import org.apache.submarine.commons.runtime.exception.SubmarineException;
+import org.apache.submarine.commons.utils.exception.SubmarineRuntimeException;
+import org.apache.submarine.server.api.Submitter;
+import org.apache.submarine.server.api.experiment.Experiment;
+import org.apache.submarine.server.api.experiment.ExperimentId;
+import org.apache.submarine.server.api.spec.ExperimentSpec;
+import org.apache.submarine.server.experiment.database.ExperimentEntity;
+import org.apache.submarine.server.experiment.database.ExperimentService;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.Reader;
+import java.net.URISyntaxException;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
+
+public class ExperimentManagerTest {
+  private final Logger LOG = LoggerFactory.getLogger(ExperimentManagerTest.class);
+  private ExperimentManager experimentManager;
+  private Submitter mockSubmitter;
+  private ExperimentService mockService;
+  private String specFile = "/experiment/spec.json";
+  private String newSpecFile = "/experiment/new_spec.json";
+  private String resultFile = "/experiment/result.json";
+  private String statusFile = "/experiment/status.json";
+
+  private ExperimentSpec spec;
+  private ExperimentSpec newSpec;
+  private Experiment result;
+  private Experiment status;
+
+
+  @Before
+  public void init() throws SubmarineException {
+    spec = (ExperimentSpec) buildFromJsonFile(ExperimentSpec.class, specFile);
+    newSpec = (ExperimentSpec) buildFromJsonFile(ExperimentSpec.class, newSpecFile);
+    result = (Experiment) buildFromJsonFile(Experiment.class, resultFile);
+    status = (Experiment) buildFromJsonFile(Experiment.class, statusFile);
+
+    mockSubmitter = mock(Submitter.class);
+    mockService = mock(ExperimentService.class);
+    experimentManager = new ExperimentManager(mockSubmitter, mockService);
+  }
+
+  @Test
+  public void testCreateExperiment() {
+
+    // Create a experimentID for this experiment
+    ExperimentId experimentId = new ExperimentId();
+    experimentId.setServerTimestamp(System.currentTimeMillis());
+    experimentId.setId(1);
+
+    // Construct expected result
+    Experiment expectedExperiment = new Experiment();
+    expectedExperiment.setSpec(spec);
+    expectedExperiment.setExperimentId(experimentId);
+    expectedExperiment.rebuild(result);
+
+    // Spy experimentManager in order to stub generateExperimentId()
+    ExperimentManager spyExperimentManager = spy(experimentManager);
+    doReturn(experimentId).when(spyExperimentManager).generateExperimentId();
+
+    // Stub mockSubmitter createExperiment
+    when(mockSubmitter.createExperiment(any(ExperimentSpec.class))).thenReturn(result);
+
+    // actual experiment should == expected experiment
+    Experiment actualExperiment = spyExperimentManager.createExperiment(spec);
+
+    verifyResult(expectedExperiment, actualExperiment);
+  }
+
+  @Test
+  public void testGetExperiment() {
+
+    // Create the experimentID for this experiment
+    ExperimentId experimentId = new ExperimentId();
+    experimentId.setServerTimestamp(System.currentTimeMillis());
+    experimentId.setId(1);
+
+    // Create the entity
+    ExperimentEntity entity = new ExperimentEntity();
+    entity.setExperimentSpec(toJson(spec));
+    entity.setId(experimentId.toString());
+
+    // Construct expected result
+    Experiment expectedExperiment = new Experiment();
+    expectedExperiment.setSpec(spec);
+    expectedExperiment.setExperimentId(experimentId);
+    expectedExperiment.rebuild(result);
+
+
+    // Stub service select
+    // Pretend there is a entity in db
+    when(mockService.select(any(String.class))).thenReturn(entity);
+
+    // Stub mockSubmitter findExperiment
+    when(mockSubmitter.findExperiment(any(ExperimentSpec.class))).thenReturn(result);
+
+    // get experiment
+    Experiment actualExperiment = experimentManager.getExperiment(experimentId.toString());
+
+    verifyResult(expectedExperiment, actualExperiment);
+  }
+
+  @Test
+  public void testPatchExperiment() {
+
+    // Create the experimentID for this experiment
+    ExperimentId experimentId = new ExperimentId();
+    experimentId.setServerTimestamp(System.currentTimeMillis());
+    experimentId.setId(1);
+
+    // Create the entity
+    ExperimentEntity entity = new ExperimentEntity();
+    entity.setExperimentSpec(toJson(spec));
+    entity.setId(experimentId.toString());
+
+    // Construct expected result
+    Experiment expectedExperiment = new Experiment();
+    expectedExperiment.setSpec(newSpec);
+    expectedExperiment.setExperimentId(experimentId);
+    expectedExperiment.rebuild(result);
+
+
+    // Stub service select
+    // Pretend there is a entity in db
+    when(mockService.select(any(String.class))).thenReturn(entity);
+
+    // Stub mockSubmitter patchExperiment
+    when(mockSubmitter.patchExperiment(any(ExperimentSpec.class))).thenReturn(result);
+
+    // patch experiment
+    Experiment actualExperiment = experimentManager.patchExperiment(experimentId.toString(), newSpec);
+
+    verifyResult(expectedExperiment, actualExperiment);
+  }
+
+  @Test
+  public void testDeleteExperiment() {
+
+    // Create the experimentID for this experiment
+    ExperimentId experimentId = new ExperimentId();
+    experimentId.setServerTimestamp(System.currentTimeMillis());
+    experimentId.setId(1);
+
+    // Create the entity
+    ExperimentEntity entity = new ExperimentEntity();
+    entity.setExperimentSpec(toJson(spec));
+    entity.setId(experimentId.toString());
+
+    // Construct expected result
+    Experiment expectedExperiment = new Experiment();
+    expectedExperiment.setSpec(spec);
+    expectedExperiment.setExperimentId(experimentId);
+    expectedExperiment.rebuild(status);
+
+
+    // Stub service select
+    // Pretend there is a entity in db
+    when(mockService.select(any(String.class))).thenReturn(entity);
+
+    // Stub mockSubmitter deleteExperiment
+    when(mockSubmitter.deleteExperiment(any(ExperimentSpec.class))).thenReturn(status);
+
+    // delete experiment
+    Experiment actualExperiment = experimentManager.deleteExperiment(experimentId.toString());
+
+    verifyResult(expectedExperiment, actualExperiment);
+  }
+
+  @Test(expected = SubmarineRuntimeException.class)
+  public void testGetNotFoundExperiment() {
+    // Create the experimentID for this experiment
+    ExperimentId experimentId = new ExperimentId();
+    experimentId.setServerTimestamp(System.currentTimeMillis());
+    experimentId.setId(1);
+
+    // Stub service select
+    // Pretend that we cannot find the entity
+    when(mockService.select(any(String.class))).thenReturn(null);
+
+    // get experiment
+    experimentManager.getExperiment(experimentId.toString());
+  }
+
+  private void verifyResult(Experiment expected, Experiment actual) {
+    assertEquals(expected.getUid(), actual.getUid());
+    assertEquals(expected.getCreatedTime(), actual.getCreatedTime());
+    assertEquals(expected.getRunningTime(), actual.getRunningTime());
+    assertEquals(expected.getAcceptedTime(), actual.getAcceptedTime());
+    assertEquals(expected.getName(), actual.getName());
+    assertEquals(expected.getStatus(), actual.getStatus());
+    assertEquals(expected.getExperimentId(), actual.getExperimentId());
+    assertEquals(expected.getFinishedTime(), actual.getFinishedTime());
+    assertEquals(expected.getSpec().getMeta().getName(), actual.getSpec().getMeta().getName());
+    assertEquals(expected.getSpec().getMeta().getFramework(), actual.getSpec().getMeta().getFramework());
+    assertEquals(expected.getSpec().getMeta().getNamespace(), actual.getSpec().getMeta().getNamespace());
+    assertEquals(
+        expected.getSpec().getEnvironment().getImage(),
+        actual.getSpec().getEnvironment().getImage())
+    ;
+  }
+
+  private Object buildFromJsonFile(Object obj, String filePath) throws SubmarineException {
+    Gson gson = new GsonBuilder().create();
+    try (Reader reader = Files.newBufferedReader(getCustomJobSpecFile(filePath).toPath(),
+      StandardCharsets.UTF_8)) {
+      if (obj.equals(ExperimentSpec.class)) {
+        return gson.fromJson(reader, ExperimentSpec.class);
+      } else {
+        return gson.fromJson(reader, Experiment.class);
+      }
+    } catch (Exception e) {
+      LOG.error(e.getMessage());
+      throw new SubmarineException(e.getMessage());
+    }
+  }
+
+  private File getCustomJobSpecFile(String path) throws URISyntaxException {
+    URL fileUrl = this.getClass().getResource(path);
+    return new File(fileUrl.toURI());
+  }
+
+  private <T> String toJson(T object) {
+    return new GsonBuilder().disableHtmlEscaping().create().toJson(object);
+  }
+}
diff --git a/submarine-server/server-core/src/test/java/org/apache/submarine/server/experiment/database/ExperimentServiceTest.java b/submarine-server/server-core/src/test/java/org/apache/submarine/server/experiment/database/ExperimentServiceTest.java
new file mode 100644
index 0000000..c6d9d0e
--- /dev/null
+++ b/submarine-server/server-core/src/test/java/org/apache/submarine/server/experiment/database/ExperimentServiceTest.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.experiment.database;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class ExperimentServiceTest {
+  private static final Logger LOG = LoggerFactory.getLogger(ExperimentServiceTest.class);
+  ExperimentService experimentService = new ExperimentService();
+
+  @After
+  public void cleanExperimentTable() throws Exception {
+    List<ExperimentEntity> entities = experimentService.selectAll();
+    for (ExperimentEntity entity: entities) {
+      experimentService.delete(entity.getId());
+    }
+  }
+
+  @Test
+  public void testInsert() throws Exception {
+    ExperimentEntity entity = new ExperimentEntity();
+    String id = "experiment_1230";
+    String spec = "{\"value\": 1}";
+
+    entity.setId(id);
+    entity.setExperimentSpec(spec);
+
+    experimentService.insert(entity);
+
+    ExperimentEntity entitySelected = experimentService.select(id);
+
+    compareEntity(entity, entitySelected);
+  }
+
+  @Test
+  public void testSelectAll() throws Exception  {
+    final int SIZE = 3;
+    List<ExperimentEntity> entities = new ArrayList<ExperimentEntity>();
+
+    for (int i = 0; i < SIZE; i++) {
+      ExperimentEntity entity = new ExperimentEntity();
+      entity.setId(String.format("experiment_%d", i));
+      entity.setExperimentSpec(String.format("{\"value\": %d}", i));
+      experimentService.insert(entity);
+      entities.add(entity);
+    }
+
+    List<ExperimentEntity> entities_selected = experimentService.selectAll();
+
+    Assert.assertEquals(SIZE, entities_selected.size());
+    for (int i = 0; i < entities_selected.size(); i++) {
+      compareEntity(entities.get(i), entities_selected.get(i));
+    }
+  };
+
+  @Test
+  public void testUpdate() throws Exception {
+    ExperimentEntity entity = new ExperimentEntity();
+    String id = "experiment_1230";
+    String spec = "{\"value\": 1}";
+    entity.setId(id);
+    entity.setExperimentSpec(spec);
+    experimentService.insert(entity);
+
+    String new_spec = "{\"value\": 2}";
+    entity.setExperimentSpec(new_spec);
+    experimentService.update(entity);
+
+    ExperimentEntity entitySelected = experimentService.select(id);
+    compareEntity(entity, entitySelected);
+  };
+
+  @Test
+  public void testDelete() throws Exception {
+    ExperimentEntity entity = new ExperimentEntity();
+    String id = "experiment_1230";
+    String spec = "{\"value\": 1}";
+
+    entity.setId(id);
+    entity.setExperimentSpec(spec);
+
+    experimentService.insert(entity);
+
+    experimentService.delete(id);
+
+    List<ExperimentEntity> entitySelected = experimentService.selectAll();
+
+    Assert.assertEquals(0, entitySelected.size());
+  };
+
+  private void compareEntity(ExperimentEntity expected, ExperimentEntity actual) {
+    Assert.assertEquals(expected.getId(), actual.getId());
+    Assert.assertEquals(expected.getExperimentSpec(), actual.getExperimentSpec());
+  }
+}
diff --git a/submarine-server/server-core/src/test/resources/experiment/new_spec.json b/submarine-server/server-core/src/test/resources/experiment/new_spec.json
new file mode 100644
index 0000000..400d69a
--- /dev/null
+++ b/submarine-server/server-core/src/test/resources/experiment/new_spec.json
@@ -0,0 +1,20 @@
+{
+  "meta": {
+    "name": "tensorflow-dist-mnist-new",
+    "namespace": "submarine",
+    "framework": "TensorFlow",
+    "cmd": "python /var/tf_mnist/mnist_with_summaries.py --log_dir=/train/log --learning_rate=0.01 --batch_size=150",
+    "envVars": {
+      "ENV_1": "ENV_1"
+    }
+  },
+  "environment": {
+    "image": "apache/submarine:tf-mnist-with-summaries-1.0"
+  },
+  "spec": {
+    "Worker": {
+      "replicas": 1,
+      "resources": "cpu=1,memory=1024M"
+    }
+  }
+}
diff --git a/submarine-server/server-core/src/test/resources/experiment/result.json b/submarine-server/server-core/src/test/resources/experiment/result.json
new file mode 100644
index 0000000..b6b615a
--- /dev/null
+++ b/submarine-server/server-core/src/test/resources/experiment/result.json
@@ -0,0 +1,7 @@
+{
+  "name":"tensorflow-dist-mnist",
+  "uid":"c9f6bf7a-5bc2-11eb-8f96-0242a5374496",
+  "status":"Created",
+  "acceptedTime":"2021-01-21T16:29:33.000+08:00",
+  "createdTime":"2021-01-21T16:29:33.000+08:00"
+}
diff --git a/submarine-server/server-core/src/test/resources/experiment/spec.json b/submarine-server/server-core/src/test/resources/experiment/spec.json
new file mode 100644
index 0000000..3a071f4
--- /dev/null
+++ b/submarine-server/server-core/src/test/resources/experiment/spec.json
@@ -0,0 +1,20 @@
+{
+  "meta": {
+    "name": "tensorflow-dist-mnist",
+    "namespace": "submarine",
+    "framework": "TensorFlow",
+    "cmd": "python /var/tf_mnist/mnist_with_summaries.py --log_dir=/train/log --learning_rate=0.01 --batch_size=150",
+    "envVars": {
+      "ENV_1": "ENV1"
+    }
+  },
+  "environment": {
+    "image": "apache/submarine:tf-mnist-with-summaries-1.0"
+  },
+  "spec": {
+    "Worker": {
+      "replicas": 1,
+      "resources": "cpu=1,memory=1024M"
+    }
+  }
+}
diff --git a/submarine-server/server-core/src/test/resources/experiment/status.json b/submarine-server/server-core/src/test/resources/experiment/status.json
new file mode 100644
index 0000000..11a22e3
--- /dev/null
+++ b/submarine-server/server-core/src/test/resources/experiment/status.json
@@ -0,0 +1,5 @@
+{
+  "name":"tensorflow-dist-mnist",
+  "uid":"c9f6bf7a-5bc2-11eb-8f96-0242a5374496",
+  "status":"Deleted"
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@submarine.apache.org
For additional commands, e-mail: dev-help@submarine.apache.org