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 2022/01/05 08:38:45 UTC

[submarine] branch master updated: SUBMARINE-1139. add internal api for status sync

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 aa92203  SUBMARINE-1139. add internal api for status sync
aa92203 is described below

commit aa92203c4fa7b35532d3ea715f736f29ce801ba6
Author: FatalLin <fa...@gmail.com>
AuthorDate: Wed Jan 5 16:21:12 2022 +0800

    SUBMARINE-1139. add internal api for status sync
    
    ### What is this PR for?
    I added an internal service in submarine-server which allow the agent which is appended on our custom resource object to sync their tracking result to database, also push the status change via web-socket.
    
    ### What type of PR is it?
    Feature
    
    ### Todos
    * [ ] - Task
    
    ### What is the Jira issue?
    https://issues.apache.org/jira/browse/SUBMARINE-1139
    ### How should this be tested?
    the unit test should be added later.
    ### Screenshots (if appropriate)
    ![flow chart](https://user-images.githubusercontent.com/5687317/147484889-45717b97-e3c9-4bd5-9614-c273e81d40c0.jpeg)
    
    ### Questions:
    * Do the license files need updating? No
    * Are there breaking changes for older versions? No
    * Does this need new documentation? No
    
    Author: FatalLin <fa...@gmail.com>
    Author: Kevin Su <pi...@apache.org>
    
    Signed-off-by: Kevin <pi...@apache.org>
    
    Closes #848 from FatalLin/SUBMARINE-1139 and squashes the following commits:
    
    9d480ed3 [FatalLin] Merge pull request #1 from pingsutw/SUBMARINE-1139
    a73179d4 [Kevin Su] Fixed typo
    63bd051e [Kevin Su] Fixed test
    e406764d [FatalLin] licence
    6b526987 [FatalLin] add unit test and modify couple places for it
    cb78d28d [FatalLin] fix ut
    aff3dc75 [FatalLin] remove duplicated parameter
    f00b48df [FatalLin] rollback developing codes
    f1504cc8 [FatalLin] add license announcement
    2331db7e [FatalLin] add license announcement
    13d1483a [FatalLin] add internal api for status sync
    8f3e4586 [FatalLin] Merge branch 'master' of https://github.com/apache/submarine into SUBMARINE-1139
    f814549e [FatalLin] add internal api for status sync
---
 conf/submarine-site.xml                            |  6 ++
 dev-support/database/submarine.sql                 |  2 +
 dev-support/docker-images/agent/Dockerfile         |  1 -
 dev-support/docker-images/agent/build.sh           | 13 +---
 .../submarine/commons/utils/SubmarineConfVars.java |  1 +
 .../commons/utils/SubmarineConfiguration.java      |  4 +
 .../org/apache/submarine/server/api/Submitter.java |  3 +-
 .../server/api/common}/CustomResourceType.java     | 20 +++--
 .../database/entity/ExperimentEntity.java          | 13 +++-
 .../server/internal/InternalServiceManager.java    | 79 +++++++++++++++++++
 .../submarine/server/notebook/NotebookManager.java |  2 +-
 .../notebook/database/entity/NotebookEntity.java   | 13 +++-
 .../notebook/database/service/NotebookService.java |  2 +
 .../server/rest/InternalServiceRestApi.java        | 91 ++++++++++++++++++++++
 .../submarine/server/rest/RestConstants.java       |  8 ++
 .../database/mappers/ExperimentMapper.xml          | 10 ++-
 .../submarine/database/mappers/NotebookMapper.xml  | 11 ++-
 .../experiment/database/ExperimentServiceTest.java |  3 +
 .../internal/InternalServiceManagerTest.java       | 83 ++++++++++++++++++++
 .../server/rest/InternalServiceRestApiTest.java    | 57 ++++++++++++++
 .../submarine/server/k8s/agent/HandlerFactory.java |  4 +-
 .../submarine/server/k8s/agent/SubmarineAgent.java | 38 +++++----
 .../k8s/agent/handler/CustomResourceHandler.java   |  9 ++-
 .../server/k8s/agent/util/RestClient.java          | 59 ++++++++++++++
 .../server/submitter/k8s/K8sSubmitter.java         |  2 +-
 .../submitter/k8s/parser/ExperimentSpecParser.java |  2 +-
 26 files changed, 482 insertions(+), 54 deletions(-)

diff --git a/conf/submarine-site.xml b/conf/submarine-site.xml
index a742d47..d26262e 100755
--- a/conf/submarine-site.xml
+++ b/conf/submarine-site.xml
@@ -161,4 +161,10 @@
     <description>The submitter which you want used in the server. Build-in k8s</description>
   </property>
 
+  <property>
+    <name>submarine.server.service.name</name>
+    <value>submarine-server</value>
+    <description>The name of submarine server deployment</description>
+  </property>
+
 </configuration>
diff --git a/dev-support/database/submarine.sql b/dev-support/database/submarine.sql
index a2a89e6..9dba21a 100644
--- a/dev-support/database/submarine.sql
+++ b/dev-support/database/submarine.sql
@@ -247,6 +247,7 @@ CREATE TABLE `experiment` (
   `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',
+  `experiment_status` varchar(20) DEFAULT NULL COMMENT 'experiment status',
    PRIMARY KEY `id` (`id`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8;
 
@@ -261,6 +262,7 @@ CREATE TABLE `notebook` (
   `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',
+  `notebook_status` varchar(20) DEFAULT NULL COMMENT 'notebook status',
    PRIMARY KEY `id` (`id`)
 ) ENGINE=InnoDB DEFAULT CHARSET=utf8;
 
diff --git a/dev-support/docker-images/agent/Dockerfile b/dev-support/docker-images/agent/Dockerfile
index 873a185..34fd4b4 100644
--- a/dev-support/docker-images/agent/Dockerfile
+++ b/dev-support/docker-images/agent/Dockerfile
@@ -35,7 +35,6 @@ ENV JAVA_HOME /usr/lib/jvm/java-1.8-openjdk/jre
 ADD ./tmp/submarine-k8s-agent-*.tar.gz /opt/
 RUN ln -s /opt/submarine-k8s-agent-* "/opt/submarine-current"
 ADD ./tmp/submarine-site.xml "/opt/submarine-current/conf/"
-ADD ./tmp/mysql-connector-java-5.1.39.jar "/opt/submarine-current/lib/"
 
 WORKDIR /opt/submarine-current
 
diff --git a/dev-support/docker-images/agent/build.sh b/dev-support/docker-images/agent/build.sh
index 8d680a8..b791b36 100755
--- a/dev-support/docker-images/agent/build.sh
+++ b/dev-support/docker-images/agent/build.sh
@@ -40,18 +40,9 @@ fi
 mkdir -p "${CURRENT_PATH}/tmp"
 cp ${SUBMARINE_HOME}/submarine-server/server-submitter/submarine-k8s-agent/target/submarine-k8s-agent-${SUBMARINE_VERSION}.tar.gz "${CURRENT_PATH}/tmp"
 
-# download mysql connect java
-MYSQL_VERSION=5.1.39
-MYSQL_JAR_URL="https://repo1.maven.org/maven2/mysql/mysql-connector-java/${MYSQL_VERSION}/mysql-connector-java-${MYSQL_VERSION}.jar"
-tmpfile=$(mktemp)
-trap "test -f $tmpfile && rm $tmpfile" RETURN
-curl -L -o $tmpfile ${MYSQL_JAR_URL}
-mv $tmpfile ${CURRENT_PATH}/tmp/mysql-connector-java-${MYSQL_VERSION}.jar
-
-# Replace the mysql jdbc.url in the submarine-site.xml file with the link name of the submarine container
-# `submarine-database` is submarine database container name
+# Replace the submarine.server.addr in the submarine-site.xml file with the link name of the submarine container
+# `submarine-server` is submarine server container name
 cp ${SUBMARINE_HOME}/conf/submarine-site.xml "${CURRENT_PATH}/tmp/"
-sed -i.bak 's/127.0.0.1:3306/submarine-database:3306/g' "${CURRENT_PATH}/tmp/submarine-site.xml"
 
 # build image
 cd ${CURRENT_PATH}
diff --git a/submarine-commons/commons-utils/src/main/java/org/apache/submarine/commons/utils/SubmarineConfVars.java b/submarine-commons/commons-utils/src/main/java/org/apache/submarine/commons/utils/SubmarineConfVars.java
index 3c5b397..ed9a35f 100644
--- a/submarine-commons/commons-utils/src/main/java/org/apache/submarine/commons/utils/SubmarineConfVars.java
+++ b/submarine-commons/commons-utils/src/main/java/org/apache/submarine/commons/utils/SubmarineConfVars.java
@@ -69,6 +69,7 @@ public class SubmarineConfVars {
     SUBMARINE_RUNTIME_CLASS("submarine.runtime.class",
         "org.apache.submarine.server.submitter.yarn.YarnRuntimeFactory"),
     SUBMARINE_SUBMITTER("submarine.submitter", "k8s"),
+    SUBMARINE_SERVER_SERVICE_NAME("submarine.server.service.name", "submarine-server"),
     ENVIRONMENT_CONDA_MIN_VERSION("environment.conda.min.version", "4.0.1"),
     ENVIRONMENT_CONDA_MAX_VERSION("environment.conda.max.version", "4.11.10");
 
diff --git a/submarine-commons/commons-utils/src/main/java/org/apache/submarine/commons/utils/SubmarineConfiguration.java b/submarine-commons/commons-utils/src/main/java/org/apache/submarine/commons/utils/SubmarineConfiguration.java
index 33ae550..3259062 100644
--- a/submarine-commons/commons-utils/src/main/java/org/apache/submarine/commons/utils/SubmarineConfiguration.java
+++ b/submarine-commons/commons-utils/src/main/java/org/apache/submarine/commons/utils/SubmarineConfiguration.java
@@ -311,6 +311,10 @@ public class SubmarineConfiguration extends XMLConfiguration {
     return getString(SubmarineConfVars.ConfVars.WORKBENCH_WEBSOCKET_MAX_TEXT_MESSAGE_SIZE);
   }
 
+  public String getServerServiceName() {
+    return getString(SubmarineConfVars.ConfVars.SUBMARINE_SERVER_SERVICE_NAME);
+  }
+  
   private String getStringValue(String name, String d) {
     String value = this.properties.get(name);
     if (value != null) {
diff --git a/submarine-server/server-api/src/main/java/org/apache/submarine/server/api/Submitter.java b/submarine-server/server-api/src/main/java/org/apache/submarine/server/api/Submitter.java
index b1066a5..2f8706e 100644
--- a/submarine-server/server-api/src/main/java/org/apache/submarine/server/api/Submitter.java
+++ b/submarine-server/server-api/src/main/java/org/apache/submarine/server/api/Submitter.java
@@ -94,10 +94,11 @@ public interface Submitter {
   /**
    * Create a notebook with spec
    * @param spec notebook spec
+   * @param notebookId notebookId
    * @return object
    * @throws SubmarineRuntimeException running error
    */
-  Notebook createNotebook(NotebookSpec spec) throws SubmarineRuntimeException;
+  Notebook createNotebook(NotebookSpec spec, String notebookId) throws SubmarineRuntimeException;
 
   /**
    * Find a notebook with spec
diff --git a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/bean/CustomResourceType.java b/submarine-server/server-api/src/main/java/org/apache/submarine/server/api/common/CustomResourceType.java
similarity index 69%
rename from submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/bean/CustomResourceType.java
rename to submarine-server/server-api/src/main/java/org/apache/submarine/server/api/common/CustomResourceType.java
index f85d634..634291e 100644
--- a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/bean/CustomResourceType.java
+++ b/submarine-server/server-api/src/main/java/org/apache/submarine/server/api/common/CustomResourceType.java
@@ -17,19 +17,17 @@
  * under the License.
  */
 
-package org.apache.submarine.server.k8s.agent.bean;
+package org.apache.submarine.server.api.common;
 
 public enum CustomResourceType {
-    TFJob("tfJob"), PYTORCHJob("pytorchJob"), Notebook("notebook");
+  TFJob("TFJob"), PYTORCHJob("PYTORCHJob"), Notebook("Notebook");
+  private String customResourceType;
     
-    private String customResourceType;
-    
-    CustomResourceType(String customResourceType) {
-        this.customResourceType = customResourceType; 
-    }
-    
-    public String getCustomResourceType() {
-        return this.customResourceType;
-    }
+  CustomResourceType(String customResourceType) {
+    this.customResourceType = customResourceType; 
+  }
     
+  public String getCustomResourceType() {
+    return this.customResourceType;
+  }
 }
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/entity/ExperimentEntity.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/entity/ExperimentEntity.java
index 560413d..2324c52 100644
--- a/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/entity/ExperimentEntity.java
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/experiment/database/entity/ExperimentEntity.java
@@ -27,6 +27,8 @@ public class ExperimentEntity extends BaseEntity {
   */
   private String experimentSpec;
 
+  private String experimentStatus;
+  
   public ExperimentEntity() {}
 
   public String getExperimentSpec() {
@@ -36,6 +38,14 @@ public class ExperimentEntity extends BaseEntity {
   public void setExperimentSpec(String experimentSpec) {
     this.experimentSpec = experimentSpec;
   }
+ 
+  public String getExperimentStatus() {
+    return experimentStatus;
+  }
+
+  public void setExperimentStatus(String experimentStatus) {
+    this.experimentStatus = experimentStatus;
+  }
 
   @Override
   public String toString() {
@@ -45,7 +55,8 @@ public class ExperimentEntity extends BaseEntity {
       ", createBy='" + createBy + '\'' +
       ", createTime=" + createTime +
       ", updateBy='" + updateBy + '\'' +
-      ", updateTime=" + updateTime +
+      ", updateTime='" + updateTime + '\'' +
+      ", experimentStatus='" + experimentStatus + "\'" +
       '}';
   }
 }
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/internal/InternalServiceManager.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/internal/InternalServiceManager.java
new file mode 100644
index 0000000..9e5cb1f
--- /dev/null
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/internal/InternalServiceManager.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.submarine.server.internal;
+
+import javax.ws.rs.core.Response.Status;
+
+import org.apache.submarine.commons.utils.exception.SubmarineRuntimeException;
+import org.apache.submarine.server.api.common.CustomResourceType;
+import org.apache.submarine.server.api.notebook.Notebook;
+import org.apache.submarine.server.experiment.database.entity.ExperimentEntity;
+import org.apache.submarine.server.experiment.database.service.ExperimentService;
+import org.apache.submarine.server.notebook.database.service.NotebookService;
+
+import com.google.common.annotations.VisibleForTesting;
+
+public class InternalServiceManager {
+  private static volatile InternalServiceManager internalServiceManager;
+    
+  private final ExperimentService experimentService;
+  private final NotebookService notebookService; 
+    
+  public static InternalServiceManager getInstance() {
+    if (internalServiceManager == null) {
+      internalServiceManager = new InternalServiceManager(new ExperimentService(), new NotebookService());
+    }
+    return internalServiceManager;
+  }
+  
+  @VisibleForTesting
+  protected InternalServiceManager(ExperimentService experimentService, NotebookService notebookService) {
+    this.experimentService = experimentService;
+    this.notebookService = notebookService;
+  }
+  
+  public boolean updateCRStatus(CustomResourceType crType, String resourceId, String status) {
+    if (crType.equals(CustomResourceType.Notebook)) {
+      return updateNotebookStatus(resourceId, status);
+    } else if (crType.equals(CustomResourceType.TFJob) || crType.equals(CustomResourceType.PYTORCHJob)) {
+      return updateExperimentStatus(resourceId, status);
+    }
+    return false;
+  }
+    
+  private boolean updateExperimentStatus(String resourceId, String status) {
+    ExperimentEntity experimentEntity = experimentService.select(resourceId);
+    if (experimentEntity == null) {
+      throw new SubmarineRuntimeException(Status.NOT_FOUND.getStatusCode(),
+        String.format("cannot find experiment with id:%s", resourceId));
+    }
+    experimentEntity.setExperimentStatus(status);
+    return experimentService.update(experimentEntity);
+  }
+    
+  private boolean updateNotebookStatus(String resourceId, String status) {
+    Notebook notebook = notebookService.select(resourceId);
+    if (notebook == null) {
+      throw new SubmarineRuntimeException(Status.NOT_FOUND.getStatusCode(),
+        String.format("cannot find notebook with id:%s", resourceId));
+    }
+    notebook.setStatus(status);
+    return notebookService.update(notebook);
+  }
+}
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/NotebookManager.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/NotebookManager.java
index db8f844..cb373cd 100644
--- a/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/NotebookManager.java
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/NotebookManager.java
@@ -92,7 +92,7 @@ public class NotebookManager {
     labels.put("notebook-owner-id", spec.getMeta().getOwnerId());
     labels.put("notebook-id", notebookId.toString());
     spec.getMeta().setLabels(labels);
-    Notebook notebook = submitter.createNotebook(spec);
+    Notebook notebook = submitter.createNotebook(spec, notebookId.toString());
 
     notebook.setNotebookId(notebookId);
     notebook.setSpec(spec);
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/database/entity/NotebookEntity.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/database/entity/NotebookEntity.java
index 21cebd9..cb83761 100644
--- a/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/database/entity/NotebookEntity.java
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/database/entity/NotebookEntity.java
@@ -26,6 +26,8 @@ public class NotebookEntity extends BaseEntity {
     Take id (inherited from BaseEntity) as the primary key for notebook table
   */
   private String notebookSpec;
+  
+  private String notebookStatus;
 
   public NotebookEntity() {
   }
@@ -37,6 +39,14 @@ public class NotebookEntity extends BaseEntity {
   public void setNotebookSpec(String notebookSpec) {
     this.notebookSpec = notebookSpec;
   }
+ 
+  public String getNotebookStatus() {
+    return notebookStatus;
+  }
+
+  public void setNotebookStatus(String noteStatus) {
+    this.notebookStatus = noteStatus;
+  }
 
   @Override
   public String toString() {
@@ -46,7 +56,8 @@ public class NotebookEntity extends BaseEntity {
         ", createBy='" + createBy + '\'' +
         ", createTime=" + createTime +
         ", updateBy='" + updateBy + '\'' +
-        ", updateTime=" + updateTime +
+        ", updateTime=" + updateTime + '\'' +
+        ", notebookStatus='" + notebookStatus + "\'" +
         '}';
   }
 }
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/database/service/NotebookService.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/database/service/NotebookService.java
index c19275a..0b63400 100644
--- a/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/database/service/NotebookService.java
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/notebook/database/service/NotebookService.java
@@ -130,6 +130,7 @@ public class NotebookService {
     try {
       entity.setId(notebook.getNotebookId().toString());
       entity.setNotebookSpec(new GsonBuilder().disableHtmlEscaping().create().toJson(notebook.getSpec()));
+      entity.setNotebookStatus(notebook.getStatus());
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
       throw new SubmarineRuntimeException("Unable to build entity from notebook");
@@ -149,6 +150,7 @@ public class NotebookService {
       notebook.setNotebookId(NotebookId.fromString(entity.getId()));
       notebook.setSpec(new Gson().fromJson(entity.getNotebookSpec(), NotebookSpec.class));
       notebook.setName(notebook.getSpec().getMeta().getName());
+      notebook.setStatus(entity.getNotebookStatus());
     } catch (Exception e) {
       LOG.error(e.getMessage(), e);
       throw new SubmarineRuntimeException("Unable to build notebook from entity");
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/rest/InternalServiceRestApi.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/rest/InternalServiceRestApi.java
new file mode 100644
index 0000000..28655d7
--- /dev/null
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/rest/InternalServiceRestApi.java
@@ -0,0 +1,91 @@
+/*
+ * 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.rest;
+
+import javax.ws.rs.Consumes;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.PathParam;
+import javax.ws.rs.Produces;
+import javax.ws.rs.core.MediaType;
+import javax.ws.rs.core.Response;
+
+import org.apache.submarine.commons.utils.exception.SubmarineRuntimeException;
+import org.apache.submarine.server.api.common.CustomResourceType;
+import org.apache.submarine.server.internal.InternalServiceManager;
+import org.apache.submarine.server.response.JsonResponse;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import io.swagger.v3.oas.annotations.Operation;
+import io.swagger.v3.oas.annotations.media.Content;
+import io.swagger.v3.oas.annotations.media.Schema;
+import io.swagger.v3.oas.annotations.responses.ApiResponse;
+
+/**
+ * Internal Service REST API v1, providing internal service to sync status of CR.
+ * 
+ */
+@Path(RestConstants.V1 + "/" + RestConstants.INTERNAL)
+@Produces("application/json")
+public class InternalServiceRestApi {
+    
+  private static final Logger LOG = LoggerFactory.getLogger(InternalServiceRestApi.class);
+  private InternalServiceManager internalServiceManager = InternalServiceManager.getInstance();
+  
+  @VisibleForTesting
+  public void setInternalServiceManager(InternalServiceManager internalServiceManager) {
+    this.internalServiceManager = internalServiceManager;
+  }
+  
+  /**
+   * Update status of custom resource
+   * @param name Name of the environment
+   * @param spec environment spec
+   * @return the detailed info about updated environment
+  */
+  @POST
+  @Path("/{customResourceType}/{resourceId}/{status}")
+  @Consumes({RestConstants.MEDIA_TYPE_YAML, MediaType.APPLICATION_JSON})
+  @Operation(summary = "Update the environment with job spec",
+          tags = {"environments"},
+          responses = {
+                  @ApiResponse(description = "successful operation", 
+                      content = @Content(
+                          schema = @Schema(
+                              implementation = String.class))),
+                  @ApiResponse(
+                      responseCode = "404", 
+                      description = "resource not found")})
+  public Response updateEnvironment(
+      @PathParam(RestConstants.CUSTOM_RESOURCE_TYPE) String type,
+      @PathParam(RestConstants.CUSTOM_RESOURCE_ID) String resourceId,
+      @PathParam(RestConstants.CUSTOM_RESOURCE_STATUS) String status) {
+    try {
+      internalServiceManager.updateCRStatus(CustomResourceType.valueOf(type), resourceId, status);
+      return new JsonResponse.Builder<String>(Response.Status.OK)
+        .success(true).build();
+    } catch (SubmarineRuntimeException e) {
+      return new JsonResponse.Builder<String>(e.getCode()).message(e.getMessage())
+        .build();
+    }
+  }
+}
diff --git a/submarine-server/server-core/src/main/java/org/apache/submarine/server/rest/RestConstants.java b/submarine-server/server-core/src/main/java/org/apache/submarine/server/rest/RestConstants.java
index 3137087..16bc29d 100644
--- a/submarine-server/server-core/src/main/java/org/apache/submarine/server/rest/RestConstants.java
+++ b/submarine-server/server-core/src/main/java/org/apache/submarine/server/rest/RestConstants.java
@@ -93,4 +93,12 @@ public class RestConstants {
    * Serve.
    */
   public static final String SERVE = "serve";
+  
+  /**
+   *  Internal
+   */
+  public static final String INTERNAL = "internal";
+  public static final String CUSTOM_RESOURCE_TYPE = "customResourceType";
+  public static final String CUSTOM_RESOURCE_ID = "resourceId";
+  public static final String CUSTOM_RESOURCE_STATUS = "status";
 }
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
index a5ae801..d455430 100644
--- 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
@@ -29,10 +29,11 @@
 
   <resultMap id="ExperimentEntityResultMap" type="org.apache.submarine.server.experiment.database.entity.ExperimentEntity" extends="BaseEntityResultMap">
     <result column="experiment_spec" jdbcType="VARCHAR" property="experimentSpec" />
+    <result column="experiment_status" property="experimentStatus"/>
   </resultMap>
 
   <sql id="Base_Column_List">
-    id, experiment_spec, create_by, create_time, update_by, update_time
+    id, experiment_spec, create_by, create_time, update_by, update_time, experiment_status
   </sql>
 
   <select id="selectAll" parameterType="java.lang.String" resultMap="ExperimentEntityResultMap">
@@ -54,16 +55,17 @@
   </delete>
 
   <insert id="insert" parameterType="org.apache.submarine.server.experiment.database.entity.ExperimentEntity">
-    insert into experiment (id, experiment_spec, create_by, create_time, update_by, update_time)
+    insert into experiment (id, experiment_spec, create_by, create_time, update_by, update_time, experiment_status)
     values (#{id,jdbcType=VARCHAR}, #{experimentSpec,jdbcType=VARCHAR},
-            #{createBy,jdbcType=VARCHAR}, now(), #{updateBy,jdbcType=VARCHAR}, now())
+            #{createBy,jdbcType=VARCHAR}, now(), #{updateBy,jdbcType=VARCHAR}, now(), #{experimentStatus,jdbcType=VARCHAR})
   </insert>
 
   <update id="update" parameterType="org.apache.submarine.server.experiment.database.entity.ExperimentEntity">
     update experiment
     <set>
-      <if test="experimentSpec != null">
+      <if test="experimentSpec != null and experimentStatus != null">
         experiment_spec = #{experimentSpec,jdbcType=VARCHAR},
+        experiment_status = #{experimentStatus, jdbcType=VARCHAR},
       </if>
       update_time = now()
     </set>
diff --git a/submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/NotebookMapper.xml b/submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/NotebookMapper.xml
index 11c85bf..39bebe1 100644
--- a/submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/NotebookMapper.xml
+++ b/submarine-server/server-core/src/main/resources/org/apache/submarine/database/mappers/NotebookMapper.xml
@@ -29,10 +29,11 @@
 
   <resultMap id="NotebookEntityResultMap" type="org.apache.submarine.server.notebook.database.entity.NotebookEntity" extends="BaseEntityResultMap">
     <result column="notebook_spec" jdbcType="VARCHAR" property="notebookSpec" />
+    <result column="notebook_status" jdbcType="VARCHAR" property="notebookStatus" />
   </resultMap>
 
   <sql id="Base_Column_List">
-    id, notebook_spec, create_by, create_time, update_by, update_time
+    id, notebook_spec, create_by, create_time, update_by, update_time, notebook_status
   </sql>
 
   <select id="selectAll" parameterType="java.lang.String" resultMap="NotebookEntityResultMap">
@@ -54,16 +55,18 @@
   </delete>
 
   <insert id="insert" parameterType="org.apache.submarine.server.notebook.database.entity.NotebookEntity">
-    insert into notebook (id, notebook_spec, create_by, create_time, update_by, update_time)
+    insert into notebook (id, notebook_spec, create_by, create_time, update_by, update_time, notebook_status)
     values (#{id,jdbcType=VARCHAR}, #{notebookSpec,jdbcType=VARCHAR},
-            #{createBy,jdbcType=VARCHAR}, now(), #{updateBy,jdbcType=VARCHAR}, now())
+            #{createBy,jdbcType=VARCHAR}, now(), #{updateBy,jdbcType=VARCHAR}, now(),
+            #{notebookStatus,jdbcType=VARCHAR})
   </insert>
 
   <update id="update" parameterType="org.apache.submarine.server.notebook.database.entity.NotebookEntity">
     update notebook
     <set>
-      <if test="notebookSpec != null">
+      <if test="notebookSpec != null and notebookStatus != null">
         notebook_spec = #{notebookSpec,jdbcType=VARCHAR},
+        notebook_status = #{notebookStatus,jdbcType=VARCHAR},
       </if>
       update_time = now()
     </set>
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
index 6377b50..92f9a7f 100644
--- 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
@@ -85,10 +85,12 @@ public class ExperimentServiceTest {
     String id = "experiment_1230";
     String spec = "{\"value\": 1}";
     entity.setId(id);
+    entity.setExperimentStatus("running");
     entity.setExperimentSpec(spec);
     experimentService.insert(entity);
 
     String new_spec = "{\"value\": 2}";
+    entity.setExperimentStatus("complete");
     entity.setExperimentSpec(new_spec);
     experimentService.update(entity);
 
@@ -117,5 +119,6 @@ public class ExperimentServiceTest {
   private void compareEntity(ExperimentEntity expected, ExperimentEntity actual) {
     Assert.assertEquals(expected.getId(), actual.getId());
     Assert.assertEquals(expected.getExperimentSpec(), actual.getExperimentSpec());
+    Assert.assertEquals(expected.getExperimentStatus(), actual.getExperimentStatus());
   }
 }
diff --git a/submarine-server/server-core/src/test/java/org/apache/submarine/server/internal/InternalServiceManagerTest.java b/submarine-server/server-core/src/test/java/org/apache/submarine/server/internal/InternalServiceManagerTest.java
new file mode 100644
index 0000000..a4575e3
--- /dev/null
+++ b/submarine-server/server-core/src/test/java/org/apache/submarine/server/internal/InternalServiceManagerTest.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   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.internal;
+
+import org.apache.submarine.commons.runtime.exception.SubmarineException;
+import org.apache.submarine.server.api.common.CustomResourceType;
+import org.apache.submarine.server.api.notebook.Notebook;
+import org.apache.submarine.server.api.notebook.NotebookId;
+import org.apache.submarine.server.api.spec.NotebookSpec;
+import org.apache.submarine.server.experiment.database.entity.ExperimentEntity;
+import org.apache.submarine.server.experiment.database.service.ExperimentService;
+import org.apache.submarine.server.notebook.database.service.NotebookService;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class InternalServiceManagerTest {
+  private final Logger LOG = LoggerFactory.getLogger(InternalServiceManagerTest.class);
+  private InternalServiceManager internalServiceManager;
+  private NotebookService notebookService;
+  private ExperimentService experimentService;
+  
+  @Before
+  public void init() throws SubmarineException {
+    notebookService = mock(NotebookService.class);
+    experimentService = mock(ExperimentService.class);
+    internalServiceManager = new InternalServiceManager(experimentService, notebookService);
+  }
+  
+  @Test
+  public void testUpdateNotebook() {
+    Notebook notebook = new Notebook();
+    notebook.setNotebookId(new NotebookId());
+    notebook.setName("test");
+    notebook.setReason("test");
+    notebook.setSpec(new NotebookSpec());
+    notebook.setStatus("running");
+    notebook.setUid("mock-user");
+    notebook.setUrl("http://submarine.org");
+      
+    when(notebookService.select(any(String.class))).thenReturn(notebook);
+    when(notebookService.update(any(Notebook.class))).thenReturn(true);
+      
+    assertEquals(internalServiceManager.updateCRStatus(CustomResourceType.Notebook,
+        notebook.getNotebookId().toString(), "complete"), true);
+  }
+  
+  @Test
+  public void testUpdateExperiment() {
+    ExperimentEntity experimentEntity = new ExperimentEntity();
+    experimentEntity.setId("test");
+    experimentEntity.setExperimentSpec("");
+    experimentEntity.setExperimentStatus("running");
+    
+    when(experimentService.select(any(String.class))).thenReturn(experimentEntity);
+    when(experimentService.update(any(ExperimentEntity.class))).thenReturn(true);
+    
+    assertEquals(internalServiceManager.updateCRStatus(CustomResourceType.TFJob,
+        experimentEntity.getId(), "complete"), true);
+  }
+}
diff --git a/submarine-server/server-core/src/test/java/org/apache/submarine/server/rest/InternalServiceRestApiTest.java b/submarine-server/server-core/src/test/java/org/apache/submarine/server/rest/InternalServiceRestApiTest.java
new file mode 100644
index 0000000..8fca2b3
--- /dev/null
+++ b/submarine-server/server-core/src/test/java/org/apache/submarine/server/rest/InternalServiceRestApiTest.java
@@ -0,0 +1,57 @@
+/*
+ * 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.rest;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import javax.ws.rs.core.Response;
+
+import org.apache.submarine.server.api.common.CustomResourceType;
+import org.apache.submarine.server.internal.InternalServiceManager;
+import org.apache.submarine.server.response.JsonResponse;
+import org.junit.Before;
+import org.junit.Test;
+
+
+public class InternalServiceRestApiTest {
+  InternalServiceRestApi internalServiceRestApi;
+
+  @Before
+  public void init() {
+    InternalServiceManager internalServiceManager = mock(InternalServiceManager.class);
+    internalServiceRestApi = mock(InternalServiceRestApi.class);
+    internalServiceRestApi.setInternalServiceManager(internalServiceManager);
+  }
+
+  @Test
+  public void testUpdateCRStatus() {
+    when(internalServiceRestApi.updateEnvironment(any(String.class),
+        any(String.class), any(String.class))).thenReturn(new JsonResponse.
+        Builder<String>(Response.Status.OK).
+        success(true).build());
+
+    Response response = internalServiceRestApi.updateEnvironment(CustomResourceType.
+            Notebook.getCustomResourceType(), "notebookId", "running");
+    assertEquals(response.getStatus(), Response.Status.OK.getStatusCode());
+  }
+}
diff --git a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/HandlerFactory.java b/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/HandlerFactory.java
index 5005284..56c676f 100644
--- a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/HandlerFactory.java
+++ b/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/HandlerFactory.java
@@ -19,7 +19,7 @@
 
 package org.apache.submarine.server.k8s.agent;
 
-import org.apache.submarine.server.k8s.agent.bean.CustomResourceType;
+import org.apache.submarine.server.api.common.CustomResourceType;
 import org.apache.submarine.server.k8s.agent.handler.CustomResourceHandler;
 
 public class HandlerFactory {
@@ -28,7 +28,7 @@ public class HandlerFactory {
     private static String HANDLER_PACKAGE = "org.apache.submarine.server.k8s.agent.handler";
     
     public static CustomResourceHandler getHandler(CustomResourceType crType) throws ClassNotFoundException, InstantiationException, IllegalAccessException {
-        String handlerClassStr = HANDLER_PACKAGE + "." +  crType.getCustomResourceType() + HANDLER_POSTFIX;
+        String handlerClassStr = HANDLER_PACKAGE + "." +  crType.toString() + HANDLER_POSTFIX;
         Class handlerClass = Class.forName(handlerClassStr);
         return (CustomResourceHandler)handlerClass.newInstance();
     }
diff --git a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/SubmarineAgent.java b/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/SubmarineAgent.java
index 1057a55..47d626d 100644
--- a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/SubmarineAgent.java
+++ b/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/SubmarineAgent.java
@@ -21,52 +21,62 @@ package org.apache.submarine.server.k8s.agent;
 
 import java.io.IOException;
 
-import org.apache.submarine.server.k8s.agent.bean.CustomResourceType;
+import org.apache.submarine.server.api.common.CustomResourceType;
 import org.apache.submarine.server.k8s.agent.handler.CustomResourceHandler;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import io.kubernetes.client.openapi.ApiClient;
-import io.kubernetes.client.openapi.Configuration;
-import io.kubernetes.client.openapi.apis.CoreV1Api;
-import io.kubernetes.client.util.Config;
-import io.kubernetes.client.util.Watch;
-import io.kubernetes.client.util.Watchable;
-import io.kubernetes.client.util.generic.GenericKubernetesApi;
-
 
 public class SubmarineAgent {
     private static final Logger LOG = LoggerFactory.getLogger(SubmarineAgent.class);
+    private String serverHost;
+    private Integer serverPort;
     private String namespace;
     private String customResourceType;
     private String customResourceName;
+    private String resourceId;
     private CustomResourceType type;
     private CustomResourceHandler handler;
     
     
-    public SubmarineAgent(String namespace, String customResourceType, String customResourceName) throws ClassNotFoundException, InstantiationException, IllegalAccessException, IOException {
+    public SubmarineAgent(String serverHost, Integer serverPort, String namespace,
+            String customResourceType, String customResourceName,
+            String resourceId) throws ClassNotFoundException,
+    InstantiationException, IllegalAccessException, IOException {
+        this.serverHost = serverHost;
+        this.serverPort = serverPort;
         this.namespace = namespace;
         this.customResourceType = customResourceType;
         this.customResourceName = customResourceName;
+        this.resourceId = resourceId;
         this.type = CustomResourceType.valueOf(customResourceType);
         this.handler = HandlerFactory.getHandler(this.type);
-
+        this.handler.init(serverHost, serverPort, namespace, customResourceType, customResourceName);
     }
     
     public void start() {
         
     }
     
-    
     public static void main(String[] args) throws ClassNotFoundException, InstantiationException, IllegalAccessException, IOException {
+        String serverHost = System.getenv("SERVER_HOST");
+        Integer serverPort = Integer.parseInt(System.getenv("SERVER_PORT"));
+        LOG.info(String.format("SERVER_HOST:%s", serverHost));
+        LOG.info(String.format("SERVER_PORT:%d", serverPort));
+        
         String namespace = System.getenv("NAMESPACE");
         String customResourceType = System.getenv("CUSTOM_RESOURCE_TYPE");
         String customResourceName = System.getenv("CUSTOM_RESOURCE_NAME");
-        LOG.info(String.format("NAMESPACE:%s", customResourceType));
+        String customResourceId = System.getenv("CUSTOM_RESOURCE_ID");
+        
+        LOG.info(String.format("NAMESPACE:%s", namespace));
         LOG.info(String.format("CUSTOM_RESOURCE_TYPE:%s", customResourceType));
         LOG.info(String.format("CUSTOM_RESOURCE_NAME:%s", customResourceName));
+        LOG.info(String.format("CUSTOM_RESOURCE_ID:%s", customResourceId));
+
+        SubmarineAgent agent = new SubmarineAgent(serverHost, serverPort, customResourceType,
+                customResourceType, customResourceName, customResourceId);
         
-        SubmarineAgent agent = new SubmarineAgent(customResourceType, customResourceType, customResourceName);
         agent.start();
         
     }
diff --git a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/handler/CustomResourceHandler.java b/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/handler/CustomResourceHandler.java
index 271ebc7..3e36908 100644
--- a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/handler/CustomResourceHandler.java
+++ b/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/handler/CustomResourceHandler.java
@@ -21,6 +21,8 @@ package org.apache.submarine.server.k8s.agent.handler;
 
 import java.io.IOException;
 
+import org.apache.submarine.server.k8s.agent.util.RestClient;
+
 import io.kubernetes.client.openapi.ApiClient;
 import io.kubernetes.client.openapi.Configuration;
 import io.kubernetes.client.openapi.apis.CoreV1Api;
@@ -32,6 +34,10 @@ public abstract class CustomResourceHandler {
     private String namespace;
     private String crType;
     private String crName;
+    private String serverHost;
+    private Integer serverPort;
+    private RestClient restClient;
+    
     
     public CustomResourceHandler() throws IOException {
         this.client = Config.defaultClient();
@@ -39,7 +45,8 @@ public abstract class CustomResourceHandler {
         this.coreApi = new CoreV1Api(this.client);
     }
     
-    public abstract void init(String namespace, String crType, String crName);
+    public abstract void init(String serverHost, Integer serverPort,
+            String namespace, String crType, String crName);
     public abstract void run();
     public abstract void onAddEvent();
     public abstract void onModifyEvent();
diff --git a/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/util/RestClient.java b/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/util/RestClient.java
new file mode 100644
index 0000000..244b867
--- /dev/null
+++ b/submarine-server/server-submitter/submarine-k8s-agent/src/main/java/org/apache/submarine/server/k8s/agent/util/RestClient.java
@@ -0,0 +1,59 @@
+/*
+ * 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.k8s.agent.util;
+
+import javax.ws.rs.client.Client;
+import javax.ws.rs.client.ClientBuilder;
+import javax.ws.rs.core.MediaType;
+
+import org.apache.submarine.commons.utils.SubmarineConfVars;
+import org.apache.submarine.commons.utils.SubmarineConfiguration;
+import org.apache.submarine.server.api.common.CustomResourceType;
+import org.apache.submarine.server.k8s.agent.SubmarineAgent;
+import org.apache.submarine.server.rest.RestConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class RestClient {
+  private static final Logger LOG = LoggerFactory.getLogger(RestClient.class);
+  private final SubmarineConfiguration conf = SubmarineConfiguration.getInstance();
+  private Client client = ClientBuilder.newClient();
+  private final String API_SERVER_URL;
+  public RestClient(String host, Integer port) {
+    LOG.info("SERVER_HOST:" + host);
+    LOG.info("SERVER_PORT:" + port);
+    API_SERVER_URL = String.format("http://%s:%d/", host, port);
+  }
+  
+    
+  public void callStatusUpdate(CustomResourceType type, String resourceId, String status) {
+      LOG.info("Targeting url:" + API_SERVER_URL);
+      LOG.info("Targeting uri:" + API_SERVER_URL);
+      
+      String uri = String.format("api/%s/%s/%s/%s/%s", RestConstants.V1,
+              RestConstants.INTERNAL, type.toString(), resourceId, status);
+      LOG.info("Targeting uri:" + uri);
+            
+      client.target(API_SERVER_URL)
+      .path(uri)
+      .request(MediaType.APPLICATION_JSON).post(null, String.class);        
+  }
+  
+}
diff --git a/submarine-server/server-submitter/submitter-k8s/src/main/java/org/apache/submarine/server/submitter/k8s/K8sSubmitter.java b/submarine-server/server-submitter/submitter-k8s/src/main/java/org/apache/submarine/server/submitter/k8s/K8sSubmitter.java
index 8ee8439..cae04f5 100644
--- a/submarine-server/server-submitter/submitter-k8s/src/main/java/org/apache/submarine/server/submitter/k8s/K8sSubmitter.java
+++ b/submarine-server/server-submitter/submitter-k8s/src/main/java/org/apache/submarine/server/submitter/k8s/K8sSubmitter.java
@@ -378,7 +378,7 @@ public class K8sSubmitter implements Submitter {
 
 
   @Override
-  public Notebook createNotebook(NotebookSpec spec) throws SubmarineRuntimeException {
+  public Notebook createNotebook(NotebookSpec spec, String notebookId) throws SubmarineRuntimeException {
     Notebook notebook;
     final String name = spec.getMeta().getName();
     final String scName = NotebookUtils.SC_NAME;
diff --git a/submarine-server/server-submitter/submitter-k8s/src/main/java/org/apache/submarine/server/submitter/k8s/parser/ExperimentSpecParser.java b/submarine-server/server-submitter/submitter-k8s/src/main/java/org/apache/submarine/server/submitter/k8s/parser/ExperimentSpecParser.java
index a5431fb..dee6cb8 100644
--- a/submarine-server/server-submitter/submitter-k8s/src/main/java/org/apache/submarine/server/submitter/k8s/parser/ExperimentSpecParser.java
+++ b/submarine-server/server-submitter/submitter-k8s/src/main/java/org/apache/submarine/server/submitter/k8s/parser/ExperimentSpecParser.java
@@ -320,7 +320,7 @@ public class ExperimentSpecParser {
     templateSpec.setSpec(podSpec);
     return templateSpec;
   }
-
+  
   private static List<V1EnvVar> parseEnvVars(ExperimentTaskSpec spec,
       Map<String, String> defaultEnvs) {
     if (spec.getEnvVars() != null) {

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