You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by jo...@apache.org on 2016/02/09 21:55:45 UTC

ambari git commit: AMBARI-14972 - Add PK to servicecomponentdesiredstate Table To Support FK Relationships (jonathanhurley)

Repository: ambari
Updated Branches:
  refs/heads/branch-dev-patch-upgrade d3db8dadd -> 697c309c1


AMBARI-14972 - Add PK to servicecomponentdesiredstate Table To Support FK Relationships (jonathanhurley)


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/697c309c
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/697c309c
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/697c309c

Branch: refs/heads/branch-dev-patch-upgrade
Commit: 697c309c19e04949f1bb9a746f5a5d1e87fb3e39
Parents: d3db8da
Author: Jonathan Hurley <jh...@hortonworks.com>
Authored: Tue Feb 9 09:37:18 2016 -0500
Committer: Jonathan Hurley <jh...@hortonworks.com>
Committed: Tue Feb 9 09:37:18 2016 -0500

----------------------------------------------------------------------
 .../dao/ServiceComponentDesiredStateDAO.java    | 57 +++++++++++--
 .../HostComponentDesiredStateEntity.java        |  2 +-
 .../orm/entities/HostComponentStateEntity.java  |  1 -
 .../ServiceComponentDesiredStateEntity.java     | 47 +++++++++--
 .../ServiceComponentDesiredStateEntityPK.java   | 84 -------------------
 .../server/state/ServiceComponentImpl.java      | 59 +++++---------
 .../svccomphost/ServiceComponentHostImpl.java   | 17 ++--
 .../server/upgrade/UpgradeCatalog150.java       | 18 +---
 .../server/upgrade/UpgradeCatalog170.java       |  9 +-
 .../server/upgrade/UpgradeCatalog200.java       |  8 +-
 .../server/upgrade/UpgradeCatalog210.java       | 69 ++++++++--------
 .../main/resources/Ambari-DDL-Derby-CREATE.sql  | 10 ++-
 .../main/resources/Ambari-DDL-MySQL-CREATE.sql  |  6 +-
 .../main/resources/Ambari-DDL-Oracle-CREATE.sql |  7 +-
 .../resources/Ambari-DDL-Postgres-CREATE.sql    | 11 ++-
 .../Ambari-DDL-Postgres-EMBEDDED-CREATE.sql     | 10 ++-
 .../resources/Ambari-DDL-SQLAnywhere-CREATE.sql |  7 +-
 .../resources/Ambari-DDL-SQLServer-CREATE.sql   | 12 ++-
 .../server/state/ServiceComponentTest.java      | 11 +--
 .../server/upgrade/UpgradeCatalog170Test.java   |  9 +-
 .../server/upgrade/UpgradeCatalog200Test.java   |  9 +-
 .../server/upgrade/UpgradeCatalog210Test.java   | 86 ++++++++++----------
 .../server/upgrade/UpgradeCatalogHelper.java    |  5 +-
 23 files changed, 270 insertions(+), 284 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ServiceComponentDesiredStateDAO.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ServiceComponentDesiredStateDAO.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ServiceComponentDesiredStateDAO.java
index 341d1fd..b8c2fcc 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ServiceComponentDesiredStateDAO.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/dao/ServiceComponentDesiredStateDAO.java
@@ -26,7 +26,6 @@ import javax.persistence.TypedQuery;
 
 import org.apache.ambari.server.orm.RequiresSession;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 
 import com.google.inject.Inject;
 import com.google.inject.Provider;
@@ -38,9 +37,22 @@ public class ServiceComponentDesiredStateDAO {
   @Inject
   Provider<EntityManager> entityManagerProvider;
 
+  /**
+   * DAO utilities for dealing mostly with {@link TypedQuery} results.
+   */
+  @Inject
+  private DaoUtils daoUtils;
+
+  /**
+   * Gets a {@link ServiceComponentDesiredStateEntity} by its PK ID.
+   *
+   * @param id
+   *          the ID.
+   * @return the entity or {@code null} if it does not exist.
+   */
   @RequiresSession
-  public ServiceComponentDesiredStateEntity findByPK(ServiceComponentDesiredStateEntityPK primaryKey) {
-    return entityManagerProvider.get().find(ServiceComponentDesiredStateEntity.class, primaryKey);
+  public ServiceComponentDesiredStateEntity findById(long id) {
+    return entityManagerProvider.get().find(ServiceComponentDesiredStateEntity.class, id);
   }
 
   @RequiresSession
@@ -55,6 +67,37 @@ public class ServiceComponentDesiredStateDAO {
     return null;
   }
 
+  /**
+   * Finds a {@link ServiceComponentDesiredStateEntity} by a combination of
+   * cluster, service, and component.
+   *
+   * @param clusterId
+   *          the cluster ID
+   * @param serviceName
+   *          the service name (not {@code null})
+   * @param componentName
+   *          the component name (not {@code null})
+   */
+  @RequiresSession
+  public ServiceComponentDesiredStateEntity findByName(long clusterId, String serviceName,
+      String componentName) {
+    EntityManager entityManager = entityManagerProvider.get();
+    TypedQuery<ServiceComponentDesiredStateEntity> query = entityManager.createNamedQuery(
+        "ServiceComponentDesiredStateEntity.findByName", ServiceComponentDesiredStateEntity.class);
+
+    query.setParameter("clusterId", clusterId);
+    query.setParameter("serviceName", serviceName);
+    query.setParameter("componentName", componentName);
+
+    ServiceComponentDesiredStateEntity entity = null;
+    List<ServiceComponentDesiredStateEntity> entities = daoUtils.selectList(query);
+    if (null != entities && !entities.isEmpty()) {
+      entity = entities.get(0);
+    }
+
+    return entity;
+  }
+
   @Transactional
   public void refresh(ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity) {
     entityManagerProvider.get().refresh(serviceComponentDesiredStateEntity);
@@ -76,8 +119,10 @@ public class ServiceComponentDesiredStateDAO {
   }
 
   @Transactional
-  public void removeByPK(ServiceComponentDesiredStateEntityPK primaryKey) {
-    ServiceComponentDesiredStateEntity entity = findByPK(primaryKey);
-    entityManagerProvider.get().remove(entity);
+  public void removeByName(long clusterId, String serviceName, String componentName) {
+    ServiceComponentDesiredStateEntity entity = findByName(clusterId, serviceName, componentName);
+    if (null != entity) {
+      entityManagerProvider.get().remove(entity);
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentDesiredStateEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentDesiredStateEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentDesiredStateEntity.java
index b57a467..fd15200 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentDesiredStateEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentDesiredStateEntity.java
@@ -20,7 +20,6 @@ package org.apache.ambari.server.orm.entities;
 import static org.apache.commons.lang.StringUtils.defaultString;
 
 import javax.persistence.Basic;
-import javax.persistence.CascadeType;
 import javax.persistence.Column;
 import javax.persistence.Entity;
 import javax.persistence.EnumType;
@@ -95,6 +94,7 @@ public class HostComponentDesiredStateEntity {
   @Column(name = "admin_state", nullable = true, insertable = true, updatable = true)
   private HostComponentAdminState adminState;
 
+  @ManyToOne
   @JoinColumns({
       @JoinColumn(name = "cluster_id", referencedColumnName = "cluster_id", nullable = false),
       @JoinColumn(name = "service_name", referencedColumnName = "service_name", nullable = false),

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java
index f1af9b0..f92f645 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/HostComponentStateEntity.java
@@ -18,7 +18,6 @@
 
 package org.apache.ambari.server.orm.entities;
 
-import javax.persistence.CascadeType;
 import javax.persistence.Column;
 import javax.persistence.Entity;
 import javax.persistence.EnumType;

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java
index bda2543..d2d1b42 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntity.java
@@ -20,35 +20,59 @@ package org.apache.ambari.server.orm.entities;
 
 import java.util.Collection;
 
-import javax.persistence.CascadeType;
 import javax.persistence.Column;
 import javax.persistence.Entity;
 import javax.persistence.EnumType;
 import javax.persistence.Enumerated;
+import javax.persistence.GeneratedValue;
+import javax.persistence.GenerationType;
 import javax.persistence.Id;
 import javax.persistence.JoinColumn;
 import javax.persistence.JoinColumns;
 import javax.persistence.ManyToOne;
+import javax.persistence.NamedQueries;
+import javax.persistence.NamedQuery;
 import javax.persistence.OneToMany;
 import javax.persistence.OneToOne;
+import javax.persistence.Table;
+import javax.persistence.TableGenerator;
+import javax.persistence.UniqueConstraint;
 
 import org.apache.ambari.server.state.State;
 
-@javax.persistence.IdClass(ServiceComponentDesiredStateEntityPK.class)
-@javax.persistence.Table(name = "servicecomponentdesiredstate")
 @Entity
+@Table(
+    name = "servicecomponentdesiredstate",
+    uniqueConstraints = @UniqueConstraint(
+        name = "unq_scdesiredstate_name",
+        columnNames = { "component_name", "service_name", "cluster_id" }) )
+@TableGenerator(
+    name = "servicecomponentdesiredstate_id_generator",
+    table = "ambari_sequences",
+    pkColumnName = "sequence_name",
+    valueColumnName = "sequence_value",
+    pkColumnValue = "servicecomponentdesiredstate_id_seq",
+    initialValue = 0)
+@NamedQueries({
+ @NamedQuery(
+    name = "ServiceComponentDesiredStateEntity.findByName",
+    query = "SELECT scds FROM ServiceComponentDesiredStateEntity scds WHERE scds.clusterId = :clusterId AND scds.serviceName = :serviceName AND scds.componentName = :componentName") })
 public class ServiceComponentDesiredStateEntity {
 
-  @Column(name = "cluster_id", nullable = false, insertable = false, updatable = false, length = 10)
   @Id
+  @Column(name = "id", nullable = false, insertable = true, updatable = false)
+  @GeneratedValue(
+      strategy = GenerationType.TABLE,
+      generator = "servicecomponentdesiredstate_id_generator")
+  private Long id;
+
+  @Column(name = "cluster_id", nullable = false, insertable = false, updatable = false, length = 10)
   private Long clusterId;
 
   @Column(name = "service_name", nullable = false, insertable = false, updatable = false)
-  @Id
   private String serviceName;
 
   @Column(name = "component_name", nullable = false, insertable = true, updatable = true)
-  @Id
   private String componentName;
 
   @Column(name = "desired_state", nullable = false, insertable = true, updatable = true)
@@ -72,6 +96,10 @@ public class ServiceComponentDesiredStateEntity {
   @OneToMany(mappedBy = "serviceComponentDesiredStateEntity")
   private Collection<HostComponentDesiredStateEntity> hostComponentDesiredStateEntities;
 
+  public Long getId() {
+    return id;
+  }
+
   public Long getClusterId() {
     return clusterId;
   }
@@ -117,12 +145,16 @@ public class ServiceComponentDesiredStateEntity {
     if (this == o) {
       return true;
     }
+
     if (o == null || getClass() != o.getClass()) {
       return false;
     }
 
     ServiceComponentDesiredStateEntity that = (ServiceComponentDesiredStateEntity) o;
 
+    if (id != null ? !id.equals(that.id) : that.id != null) {
+      return false;
+    }
     if (clusterId != null ? !clusterId.equals(that.clusterId) : that.clusterId != null) {
       return false;
     }
@@ -144,7 +176,8 @@ public class ServiceComponentDesiredStateEntity {
 
   @Override
   public int hashCode() {
-    int result = clusterId != null ? clusterId.intValue() : 0;
+    int result = id != null ? id.hashCode() : 0;
+    result = 31 * result + (clusterId != null ? clusterId.hashCode() : 0);
     result = 31 * result + (serviceName != null ? serviceName.hashCode() : 0);
     result = 31 * result + (componentName != null ? componentName.hashCode() : 0);
     result = 31 * result + (desiredState != null ? desiredState.hashCode() : 0);

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntityPK.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntityPK.java b/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntityPK.java
deleted file mode 100644
index d56e555..0000000
--- a/ambari-server/src/main/java/org/apache/ambari/server/orm/entities/ServiceComponentDesiredStateEntityPK.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ambari.server.orm.entities;
-
-import javax.persistence.Column;
-import javax.persistence.Id;
-import java.io.Serializable;
-
-@SuppressWarnings("serial")
-public class ServiceComponentDesiredStateEntityPK implements Serializable {
-  private Long clusterId;
-
-  @Column(name = "cluster_id", nullable = false, insertable = true, updatable = true, length = 10)
-  @Id
-  public Long getClusterId() {
-    return clusterId;
-  }
-
-  public void setClusterId(Long clusterId) {
-    this.clusterId = clusterId;
-  }
-
-  private String serviceName;
-
-  @javax.persistence.Column(name = "service_name", nullable = false, insertable = false, updatable = false)
-  @Id
-  public String getServiceName() {
-    return serviceName;
-  }
-
-  public void setServiceName(String serviceName) {
-    this.serviceName = serviceName;
-  }
-
-  private String componentName;
-
-  @Id
-  @Column(name = "component_name", nullable = false, insertable = true, updatable = true)
-  public String getComponentName() {
-    return componentName;
-  }
-
-  public void setComponentName(String componentName) {
-    this.componentName = componentName;
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) return true;
-    if (o == null || getClass() != o.getClass()) return false;
-
-    ServiceComponentDesiredStateEntityPK that = (ServiceComponentDesiredStateEntityPK) o;
-
-    if (clusterId != null ? !clusterId.equals(that.clusterId) : that.clusterId != null) return false;
-    if (componentName != null ? !componentName.equals(that.componentName) : that.componentName != null) return false;
-    if (serviceName != null ? !serviceName.equals(that.serviceName) : that.serviceName != null) return false;
-
-    return true;
-  }
-
-  @Override
-  public int hashCode() {
-    int result = clusterId != null ? clusterId.intValue() : 0;
-    result = 31 * result + (serviceName != null ? serviceName.hashCode() : 0);
-    result = 31 * result + (componentName != null ? componentName.hashCode() : 0);
-    return result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceComponentImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceComponentImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceComponentImpl.java
index 7e1dd1d..4afc857 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceComponentImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/ServiceComponentImpl.java
@@ -39,7 +39,6 @@ import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntity;
 import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.StackEntity;
 import org.apache.ambari.server.state.cluster.ClusterImpl;
 import org.slf4j.Logger;
@@ -74,8 +73,8 @@ public class ServiceComponentImpl implements ServiceComponent {
   private ServiceComponentHostFactory serviceComponentHostFactory;
   @Inject
   private AmbariMetaInfo ambariMetaInfo;
-  private ServiceComponentDesiredStateEntity desiredStateEntity;
-  private ServiceComponentDesiredStateEntityPK desiredStateEntityPK;
+
+  ServiceComponentDesiredStateEntity desiredStateEntity;
   private Map<String, ServiceComponentHost> hostComponents;
 
   /**
@@ -90,12 +89,13 @@ public class ServiceComponentImpl implements ServiceComponent {
     injector.injectMembers(this);
     clusterGlobalLock = service.getClusterGlobalLock();
     this.service = service;
-    desiredStateEntity = new ServiceComponentDesiredStateEntity(  );
+
+    desiredStateEntity = new ServiceComponentDesiredStateEntity();
     desiredStateEntity.setComponentName(componentName);
     desiredStateEntity.setDesiredState(State.INIT);
     desiredStateEntity.setServiceName(service.getName());
     desiredStateEntity.setClusterId(service.getClusterId());
-    desiredStateEntityPK = getDesiredStateEntityPK(desiredStateEntity);
+
     setDesiredStackVersion(service.getDesiredStackVersion());
 
     hostComponents = new HashMap<String, ServiceComponentHost>();
@@ -125,11 +125,12 @@ public class ServiceComponentImpl implements ServiceComponent {
     injector.injectMembers(this);
     clusterGlobalLock = service.getClusterGlobalLock();
     this.service = service;
+
     desiredStateEntity = serviceComponentDesiredStateEntity;
-    this.componentName = serviceComponentDesiredStateEntity.getComponentName();
+    componentName = serviceComponentDesiredStateEntity.getComponentName();
 
     hostComponents = new HashMap<String, ServiceComponentHost>();
-    for (HostComponentStateEntity hostComponentStateEntity : desiredStateEntity.getHostComponentStateEntities()) {
+    for (HostComponentStateEntity hostComponentStateEntity : serviceComponentDesiredStateEntity.getHostComponentStateEntities()) {
       HostComponentDesiredStateEntityPK pk = new HostComponentDesiredStateEntityPK();
       pk.setClusterId(hostComponentStateEntity.getClusterId());
       pk.setServiceName(hostComponentStateEntity.getServiceName());
@@ -167,8 +168,6 @@ public class ServiceComponentImpl implements ServiceComponent {
           + ", stackInfo=" + stackId.getStackId());
     }
 
-    desiredStateEntityPK = getDesiredStateEntityPK(desiredStateEntity);
-
     persisted = true;
   }
 
@@ -179,15 +178,7 @@ public class ServiceComponentImpl implements ServiceComponent {
 
   @Override
   public String getName() {
-    ServiceComponentDesiredStateEntity desiredStateEntity = getDesiredStateEntity();
-    if (desiredStateEntity != null) {
-      return desiredStateEntity.getComponentName();
-    } else {
-      LOG.warn("Trying to fetch a member from an entity object that may " +
-        "have been previously deleted, serviceName = " + getServiceName() + ", " +
-        "componentName = " + componentName);
-    }
-    return null;
+    return componentName;
   }
 
   @Override
@@ -372,7 +363,7 @@ public class ServiceComponentImpl implements ServiceComponent {
       ServiceComponentDesiredStateEntity desiredStateEntity = getDesiredStateEntity();
       if (desiredStateEntity != null) {
         desiredStateEntity.setDesiredState(state);
-        saveIfPersisted();
+        saveIfPersisted(desiredStateEntity);
       } else {
         LOG.warn("Setting a member on an entity object that may have been " +
           "previously deleted, serviceName = " + (service != null ? service.getName() : ""));
@@ -418,7 +409,7 @@ public class ServiceComponentImpl implements ServiceComponent {
       ServiceComponentDesiredStateEntity desiredStateEntity = getDesiredStateEntity();
       if (desiredStateEntity != null) {
         desiredStateEntity.setDesiredStack(stackEntity);
-        saveIfPersisted();
+        saveIfPersisted(desiredStateEntity);
       } else {
         LOG.warn("Setting a member on an entity object that may have been " +
           "previously deleted, serviceName = " + (service != null ? service.getName() : ""));
@@ -517,7 +508,7 @@ public class ServiceComponentImpl implements ServiceComponent {
           // service.refresh();
           persisted = true;
         } else {
-          saveIfPersisted();
+          saveIfPersisted(desiredStateEntity);
         }
       } finally {
         readWriteLock.writeLock().unlock();
@@ -536,9 +527,11 @@ public class ServiceComponentImpl implements ServiceComponent {
     pk.setServiceName(service.getName());
     ClusterServiceEntity serviceEntity = clusterServiceDAO.findByPK(pk);
 
+    ServiceComponentDesiredStateEntity desiredStateEntity = getDesiredStateEntity();
     desiredStateEntity.setClusterServiceEntity(serviceEntity);
+
     serviceComponentDesiredStateDAO.create(desiredStateEntity);
-    clusterServiceDAO.merge(serviceEntity);
+    serviceEntity = clusterServiceDAO.merge(serviceEntity);
   }
 
   @Override
@@ -547,10 +540,6 @@ public class ServiceComponentImpl implements ServiceComponent {
     readWriteLock.writeLock().lock();
     try {
       if (isPersisted()) {
-        ServiceComponentDesiredStateEntityPK pk = new ServiceComponentDesiredStateEntityPK();
-        pk.setComponentName(getName());
-        pk.setClusterId(getClusterId());
-        pk.setServiceName(getServiceName());
         serviceComponentDesiredStateDAO.refresh(getDesiredStateEntity());
       }
     } finally {
@@ -564,9 +553,9 @@ public class ServiceComponentImpl implements ServiceComponent {
    * has already been acquired from {@link #readWriteLock}.
    */
   @Transactional
-  void saveIfPersisted() {
+  void saveIfPersisted(ServiceComponentDesiredStateEntity desiredStateEntity) {
     if (isPersisted()) {
-      serviceComponentDesiredStateDAO.merge(desiredStateEntity);
+      desiredStateEntity = serviceComponentDesiredStateDAO.merge(desiredStateEntity);
     }
   }
 
@@ -732,18 +721,10 @@ public class ServiceComponentImpl implements ServiceComponent {
 
   // Refresh cached reference after ever setter
   private ServiceComponentDesiredStateEntity getDesiredStateEntity() {
-    if (isPersisted()) {
-      desiredStateEntity = serviceComponentDesiredStateDAO.findByPK(desiredStateEntityPK);
+    if (!isPersisted()) {
+      return desiredStateEntity;
     }
-    return desiredStateEntity;
-  }
-
-  private ServiceComponentDesiredStateEntityPK getDesiredStateEntityPK(ServiceComponentDesiredStateEntity desiredStateEntity) {
-    ServiceComponentDesiredStateEntityPK pk = new ServiceComponentDesiredStateEntityPK();
-    pk.setClusterId(desiredStateEntity.getClusterId());
-    pk.setComponentName(desiredStateEntity.getComponentName());
-    pk.setServiceName(desiredStateEntity.getServiceName());
 
-    return pk;
+    return serviceComponentDesiredStateDAO.findById(desiredStateEntity.getId());
   }
 }

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
index bfb6214..92828af 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/svccomphost/ServiceComponentHostImpl.java
@@ -51,7 +51,6 @@ import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
 import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.RepositoryVersionEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.StackEntity;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
@@ -1453,26 +1452,26 @@ public class ServiceComponentHostImpl implements ServiceComponentHost {
 
   @Transactional
   protected void persistEntities() {
+    ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity = serviceComponentDesiredStateDAO.findByName(
+        serviceComponent.getClusterId(), serviceComponent.getServiceName(),
+        serviceComponent.getName());
+
     HostEntity hostEntity = hostDAO.findByName(getHostName());
     hostEntity.addHostComponentStateEntity(stateEntity);
     hostEntity.addHostComponentDesiredStateEntity(desiredStateEntity);
 
-    ServiceComponentDesiredStateEntityPK dpk = new ServiceComponentDesiredStateEntityPK();
-    dpk.setClusterId(serviceComponent.getClusterId());
-    dpk.setServiceName(serviceComponent.getServiceName());
-    dpk.setComponentName(serviceComponent.getName());
-
-    ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity = serviceComponentDesiredStateDAO.findByPK(dpk);
-    serviceComponentDesiredStateEntity.getHostComponentDesiredStateEntities().add(desiredStateEntity);
-
     desiredStateEntity.setServiceComponentDesiredStateEntity(serviceComponentDesiredStateEntity);
     desiredStateEntity.setHostEntity(hostEntity);
+
     stateEntity.setServiceComponentDesiredStateEntity(serviceComponentDesiredStateEntity);
     stateEntity.setHostEntity(hostEntity);
 
     hostComponentStateDAO.create(stateEntity);
     hostComponentDesiredStateDAO.create(desiredStateEntity);
 
+    serviceComponentDesiredStateEntity.getHostComponentDesiredStateEntities().add(
+        desiredStateEntity);
+
     HostComponentStateEntity stateEntity = hostComponentStateDAO.findByIndex(serviceComponent.getClusterId(),
       serviceComponent.getServiceName(), serviceComponent.getName(), hostEntity.getHostId());
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog150.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog150.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog150.java
index b00b0e8..ff94795 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog150.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog150.java
@@ -64,9 +64,7 @@ import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.HostRoleCommandEntity;
 import org.apache.ambari.server.orm.entities.KeyValueEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.StackEntity;
-import org.apache.ambari.server.orm.entities.UserEntity;
 import org.apache.ambari.server.state.HostComponentAdminState;
 import org.apache.ambari.server.state.PropertyInfo;
 import org.apache.ambari.server.state.ServiceInfo;
@@ -607,24 +605,16 @@ public class UpgradeCatalog150 extends AbstractUpgradeCatalog {
 
     List<ClusterEntity> clusterEntities = clusterDAO.findAll();
     for (final ClusterEntity clusterEntity : clusterEntities) {
-      ServiceComponentDesiredStateEntityPK pkHS = new ServiceComponentDesiredStateEntityPK();
-      pkHS.setComponentName("HISTORYSERVER");
-      pkHS.setClusterId(clusterEntity.getClusterId());
-      pkHS.setServiceName("MAPREDUCE");
-
-      ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntityHS = serviceComponentDesiredStateDAO.findByPK(pkHS);
+      ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntityHS = serviceComponentDesiredStateDAO.findByName(
+          clusterEntity.getClusterId(), "MAPREDUCE", "HISTORYSERVER");
 
       // already have historyserver
       if(serviceComponentDesiredStateEntityHS != null) {
         continue;
       }
 
-      ServiceComponentDesiredStateEntityPK pkJT = new ServiceComponentDesiredStateEntityPK();
-      pkJT.setComponentName("JOBTRACKER");
-      pkJT.setClusterId(clusterEntity.getClusterId());
-      pkJT.setServiceName("MAPREDUCE");
-
-      ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntityJT = serviceComponentDesiredStateDAO.findByPK(pkJT);
+      ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntityJT = serviceComponentDesiredStateDAO.findByName(
+          clusterEntity.getClusterId(), "MAPREDUCE", "JOBTRACKER");
 
       // no jobtracker present probably mapreduce is not installed
       if(serviceComponentDesiredStateEntityJT == null) {

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog170.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog170.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog170.java
index 91de82a..b3b1b70 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog170.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog170.java
@@ -86,7 +86,6 @@ import org.apache.ambari.server.orm.entities.PrivilegeEntity;
 import org.apache.ambari.server.orm.entities.ResourceEntity;
 import org.apache.ambari.server.orm.entities.ResourceTypeEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.ServiceDesiredStateEntity;
 import org.apache.ambari.server.orm.entities.ServiceDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.StackEntity;
@@ -769,11 +768,8 @@ public class UpgradeCatalog170 extends AbstractUpgradeCatalog {
 
     List<ClusterEntity> clusterEntities = clusterDAO.findAll();
     for (final ClusterEntity clusterEntity : clusterEntities) {
-      ServiceComponentDesiredStateEntityPK pkHCATInHcatalog = new ServiceComponentDesiredStateEntityPK();
-      pkHCATInHcatalog.setComponentName(componentName);
-      pkHCATInHcatalog.setClusterId(clusterEntity.getClusterId());
-      pkHCATInHcatalog.setServiceName(serviceNameToBeDeleted);
-      ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntityToDelete = serviceComponentDesiredStateDAO.findByPK(pkHCATInHcatalog);
+      ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntityToDelete = serviceComponentDesiredStateDAO.findByName(
+          clusterEntity.getClusterId(), serviceNameToBeDeleted, componentName);
 
       if (serviceComponentDesiredStateEntityToDelete == null) {
         continue;
@@ -803,6 +799,7 @@ public class UpgradeCatalog170 extends AbstractUpgradeCatalog {
       serviceComponentDesiredStateEntity.setDesiredStack(serviceComponentDesiredStateEntityToDelete.getDesiredStack());
       serviceComponentDesiredStateEntity.setDesiredState(serviceComponentDesiredStateEntityToDelete.getDesiredState());
       serviceComponentDesiredStateEntity.setClusterServiceEntity(clusterServiceEntity);
+      serviceComponentDesiredStateDAO.create(serviceComponentDesiredStateEntity);
 
       Iterator<HostComponentDesiredStateEntity> hostComponentDesiredStateIterator = serviceComponentDesiredStateEntityToDelete.getHostComponentDesiredStateEntities().iterator();
       Iterator<HostComponentStateEntity> hostComponentStateIterator = serviceComponentDesiredStateEntityToDelete.getHostComponentStateEntities().iterator();

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog200.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog200.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog200.java
index 70b8f9f..143f9af 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog200.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog200.java
@@ -45,7 +45,6 @@ import org.apache.ambari.server.orm.entities.ClusterServiceEntityPK;
 import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntity;
 import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.ServiceDesiredStateEntity;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
@@ -516,11 +515,8 @@ public class UpgradeCatalog200 extends AbstractUpgradeCatalog {
           }
 
           // remove component state
-          ServiceComponentDesiredStateEntityPK primaryKey = new ServiceComponentDesiredStateEntityPK();
-          primaryKey.setClusterId(nagios.getClusterId());
-          primaryKey.setComponentName(componentDesiredState.getComponentName());
-          primaryKey.setServiceName(componentDesiredState.getServiceName());
-          componentDesiredStateDao.removeByPK(primaryKey);
+          componentDesiredStateDao.removeByName(nagios.getClusterId(),
+              componentDesiredState.getServiceName(), componentDesiredState.getComponentName());
         }
 
         // remove service state

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java
index faf4b96..d97c0c1 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/upgrade/UpgradeCatalog210.java
@@ -18,12 +18,28 @@
 
 package org.apache.ambari.server.upgrade;
 
-import com.google.gson.Gson;
-import com.google.gson.JsonElement;
-import com.google.gson.JsonObject;
-import com.google.gson.JsonParser;
-import com.google.inject.Inject;
-import com.google.inject.Injector;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.regex.Matcher;
+
+import javax.persistence.EntityManager;
+import javax.persistence.Query;
+import javax.persistence.criteria.CriteriaBuilder;
+import javax.persistence.criteria.CriteriaDelete;
+import javax.persistence.criteria.Root;
+
 import org.apache.ambari.server.AmbariException;
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.configuration.Configuration;
@@ -39,7 +55,6 @@ import org.apache.ambari.server.orm.entities.ArtifactEntity;
 import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntity;
 import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.StackEntity;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
@@ -58,26 +73,12 @@ import org.eclipse.persistence.internal.databaseaccess.FieldTypeDefinition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.persistence.EntityManager;
-import javax.persistence.Query;
-import javax.persistence.criteria.CriteriaBuilder;
-import javax.persistence.criteria.CriteriaDelete;
-import javax.persistence.criteria.Root;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
-import java.util.regex.Matcher;
+import com.google.gson.Gson;
+import com.google.gson.JsonElement;
+import com.google.gson.JsonObject;
+import com.google.gson.JsonParser;
+import com.google.inject.Inject;
+import com.google.inject.Injector;
 
 
 /**
@@ -1078,11 +1079,9 @@ public class UpgradeCatalog210 extends AbstractUpgradeCatalog {
             @Override
             public void run() {
             ServiceComponentDesiredStateDAO dao = injector.getInstance(ServiceComponentDesiredStateDAO.class);
-            ServiceComponentDesiredStateEntityPK entityPK = new ServiceComponentDesiredStateEntityPK();
-            entityPK.setClusterId(cluster.getClusterId());
-            entityPK.setServiceName("STORM");
-            entityPK.setComponentName("STORM_REST_API");
-            ServiceComponentDesiredStateEntity entity = dao.findByPK(entityPK);
+              ServiceComponentDesiredStateEntity entity = dao.findByName(cluster.getClusterId(),
+                  "STORM", "STORM_REST_API");
+
             if (entity != null) {
               EntityManager em = getEntityManagerProvider().get();
               CriteriaBuilder cb = em.getCriteriaBuilder();
@@ -1227,7 +1226,7 @@ public class UpgradeCatalog210 extends AbstractUpgradeCatalog {
             JsonObject reporting = rootJson.getAsJsonObject("reporting");
             JsonObject ok = reporting.getAsJsonObject("ok");
             JsonObject warning = reporting.getAsJsonObject("warning");
-            JsonObject critical = reporting.getAsJsonObject("critical");            
+            JsonObject critical = reporting.getAsJsonObject("critical");
 
             rootJson.remove("type");
             rootJson.remove("default_port");
@@ -1533,8 +1532,8 @@ public class UpgradeCatalog210 extends AbstractUpgradeCatalog {
               hiveSiteAddProps.put("hive.server2.authentication.kerberos.keytab", "");
               hiveSiteAddProps.put("hive.server2.authentication.kerberos.principal", "");
             }
-            
-            
+
+
             updateConfigurationPropertiesForCluster(cluster, "hive-site", hiveSiteAddProps, hiveSiteRemoveProps, false, true);
           }
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/resources/Ambari-DDL-Derby-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-Derby-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-Derby-CREATE.sql
index e1e2813..6c94999 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Derby-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Derby-CREATE.sql
@@ -171,14 +171,18 @@ CREATE TABLE host_version (
   PRIMARY KEY (id));
 
 CREATE TABLE servicecomponentdesiredstate (
+  id BIGINT NOT NULL,
   component_name VARCHAR(255) NOT NULL,
   cluster_id BIGINT NOT NULL,
   desired_stack_id BIGINT NOT NULL,
   desired_state VARCHAR(255) NOT NULL,
   service_name VARCHAR(255) NOT NULL,
-  PRIMARY KEY (component_name, cluster_id, service_name)
+  CONSTRAINT pk_servicecomponentdesiredstate PRIMARY KEY (id),
+  CONSTRAINT unq_scdesiredstate_name UNIQUE(component_name,service_name,cluster_id)
 );
 
+CREATE INDEX idx_sc_desired_state ON servicecomponentdesiredstate(component_name, service_name, cluster_id);
+
 CREATE TABLE servicedesiredstate (
   cluster_id BIGINT NOT NULL,
   desired_host_role_mapping INTEGER NOT NULL,
@@ -1039,7 +1043,9 @@ INSERT INTO ambari_sequences (sequence_name, sequence_value)
   union all
   select 'setting_id_seq', 0 FROM SYSIBM.SYSDUMMY1
   union all
-  select 'hostcomponentstate_id_seq', 0  FROM SYSIBM.SYSDUMMY1;
+  select 'hostcomponentstate_id_seq', 0  FROM SYSIBM.SYSDUMMY1
+  union all
+  select 'servicecomponentdesiredstate_id_seq', 0  FROM SYSIBM.SYSDUMMY1;
 
 INSERT INTO adminresourcetype (resource_type_id, resource_type_name)
   SELECT 1, 'AMBARI' FROM SYSIBM.SYSDUMMY1

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
index 46b1983..7f5eddc 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-MySQL-CREATE.sql
@@ -172,14 +172,18 @@ CREATE TABLE host_version (
   PRIMARY KEY (id));
 
 CREATE TABLE servicecomponentdesiredstate (
+  id BIGINT NOT NULL,
   component_name VARCHAR(100) NOT NULL,
   cluster_id BIGINT NOT NULL,
   desired_stack_id BIGINT NOT NULL,
   desired_state VARCHAR(255) NOT NULL,
   service_name VARCHAR(100) NOT NULL,
-  PRIMARY KEY (component_name, cluster_id, service_name)
+  CONSTRAINT pk_servicecomponentdesiredstate PRIMARY KEY (id),
+  CONSTRAINT unq_scdesiredstate_name UNIQUE(component_name,service_name,cluster_id)
 );
 
+CREATE INDEX idx_sc_desired_state ON servicecomponentdesiredstate(component_name, service_name, cluster_id);
+
 CREATE TABLE servicedesiredstate (
   cluster_id BIGINT NOT NULL,
   desired_host_role_mapping INTEGER NOT NULL,

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
index c320720..e3e4336 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Oracle-CREATE.sql
@@ -162,14 +162,18 @@ CREATE TABLE host_version (
   PRIMARY KEY (id));
 
 CREATE TABLE servicecomponentdesiredstate (
+  id NUMBER(19) NOT NULL,
   component_name VARCHAR2(255) NOT NULL,
   cluster_id NUMBER(19) NOT NULL,
   desired_stack_id NUMBER(19) NOT NULL,
   desired_state VARCHAR2(255) NOT NULL,
   service_name VARCHAR2(255) NOT NULL,
-  PRIMARY KEY (component_name, cluster_id, service_name)
+  CONSTRAINT pk_servicecomponentdesiredstate PRIMARY KEY (alert_id),
+  CONSTRAINT unq_scdesiredstate_name UNIQUE(component_name,service_name,cluster_id)
 );
 
+CREATE INDEX idx_sc_desired_state ON servicecomponentdesiredstate(component_name, service_name, cluster_id);
+
 CREATE TABLE servicedesiredstate (
   cluster_id NUMBER(19) NOT NULL,
   desired_host_role_mapping NUMBER(10) NOT NULL,
@@ -994,6 +998,7 @@ INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('topology_re
 INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('topology_host_group_id_seq', 0);
 INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('setting_id_seq', 0);
 INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('hostcomponentstate_id_seq', 0);
+INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('servicecomponentdesiredstate_id_seq', 0);
 
 INSERT INTO metainfo("metainfo_key", "metainfo_value") values ('version', '${ambariVersion}');
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
index e6e6103..f5983a5 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Postgres-CREATE.sql
@@ -171,14 +171,19 @@ CREATE TABLE host_version (
   PRIMARY KEY (id));
 
 CREATE TABLE servicecomponentdesiredstate (
+  id BIGINT NOT NULL,
   component_name VARCHAR(255) NOT NULL,
   cluster_id BIGINT NOT NULL,
   desired_stack_id BIGINT NOT NULL,
   desired_state VARCHAR(255) NOT NULL,
   service_name VARCHAR(255) NOT NULL,
-  PRIMARY KEY (component_name, cluster_id, service_name)
+  CONSTRAINT pk_servicecomponentdesiredstate PRIMARY KEY (id),
+  CONSTRAINT unq_scdesiredstate_name UNIQUE(component_name,service_name,cluster_id)
 );
 
+CREATE INDEX idx_sc_desired_state ON servicecomponentdesiredstate(component_name, service_name, cluster_id);
+
+
 CREATE TABLE servicedesiredstate (
   cluster_id BIGINT NOT NULL,
   desired_host_role_mapping INTEGER NOT NULL,
@@ -1040,7 +1045,9 @@ INSERT INTO ambari_sequences (sequence_name, sequence_value)
   union all
   select 'setting_id_seq', 0
   union all
-  select 'hostcomponentstate_id_seq', 0;
+  select 'hostcomponentstate_id_seq', 0
+  union all
+  select 'servicecomponentdesiredstate_id_seq', 0;
 
 INSERT INTO adminresourcetype (resource_type_id, resource_type_name)
   SELECT 1, 'AMBARI'

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
index 56d0947..9d9a986 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-Postgres-EMBEDDED-CREATE.sql
@@ -196,15 +196,19 @@ CREATE TABLE ambari.host_version (
 GRANT ALL PRIVILEGES ON TABLE ambari.host_version TO :username;
 
 CREATE TABLE ambari.servicecomponentdesiredstate (
+  id BIGINT NOT NULL,
   component_name VARCHAR(255) NOT NULL,
   cluster_id BIGINT NOT NULL,
   desired_stack_id BIGINT NOT NULL,
   desired_state VARCHAR(255) NOT NULL,
   service_name VARCHAR(255) NOT NULL,
-  PRIMARY KEY (component_name, cluster_id, service_name)
+  CONSTRAINT pk_servicecomponentdesiredstate PRIMARY KEY (id),
+  CONSTRAINT unq_scdesiredstate_name UNIQUE(component_name,service_name,cluster_id)
 );
 GRANT ALL PRIVILEGES ON TABLE ambari.servicecomponentdesiredstate TO :username;
 
+CREATE INDEX idx_sc_desired_state ON ambari.servicecomponentdesiredstate(component_name, service_name, cluster_id);
+
 CREATE TABLE ambari.servicedesiredstate (
   cluster_id BIGINT NOT NULL,
   desired_host_role_mapping INTEGER NOT NULL,
@@ -1139,7 +1143,9 @@ INSERT INTO ambari.ambari_sequences (sequence_name, sequence_value)
   union all
   select 'setting_id_seq', 0
   union all
-  select 'hostcomponentstate_id_seq', 0;
+  select 'hostcomponentstate_id_seq', 0
+  union all
+  select 'servicecomponentdesiredstate_id_seq', 0;
 
 INSERT INTO ambari.adminresourcetype (resource_type_id, resource_type_name)
   SELECT 1, 'AMBARI'

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql
index 542b815..8c82afc 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-SQLAnywhere-CREATE.sql
@@ -161,14 +161,18 @@ CREATE TABLE host_version (
   PRIMARY KEY (id));
 
 CREATE TABLE servicecomponentdesiredstate (
+  id NUMERIC(19) NOT NULL,
   component_name VARCHAR(255) NOT NULL,
   cluster_id NUMERIC(19) NOT NULL,
   desired_stack_id NUMERIC(19) NOT NULL,
   desired_state VARCHAR(255) NOT NULL,
   service_name VARCHAR(255) NOT NULL,
-  PRIMARY KEY (component_name, cluster_id, service_name)
+  CONSTRAINT pk_servicecomponentdesiredstate PRIMARY KEY (id),
+  CONSTRAINT unq_scdesiredstate_name UNIQUE(component_name,service_name,cluster_id)
 );
 
+CREATE INDEX idx_sc_desired_state ON servicecomponentdesiredstate(component_name, service_name, cluster_id);
+
 CREATE TABLE servicedesiredstate (
   cluster_id NUMERIC(19) NOT NULL,
   desired_host_role_mapping INTEGER NOT NULL,
@@ -991,6 +995,7 @@ INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('topology_re
 INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('topology_host_group_id_seq', 0);
 INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('setting_id_seq', 0);
 INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('hostcomponentstate_id_seq', 0);
+INSERT INTO ambari_sequences(sequence_name, sequence_value) values ('servicecomponentdesiredstate_id_seq', 0);
 
 insert into adminresourcetype (resource_type_id, resource_type_name)
   select 1, 'AMBARI'

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql b/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
index 885e422..e36519e 100644
--- a/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
+++ b/ambari-server/src/main/resources/Ambari-DDL-SQLServer-CREATE.sql
@@ -181,13 +181,18 @@ CREATE TABLE hoststate (
   PRIMARY KEY CLUSTERED (host_id));
 
 CREATE TABLE servicecomponentdesiredstate (
+  id BIGINT NOT NULL,
   component_name VARCHAR(255) NOT NULL,
   cluster_id BIGINT NOT NULL,
   desired_stack_id BIGINT NOT NULL,
   desired_state VARCHAR(255) NOT NULL,
   service_name VARCHAR(255) NOT NULL,
-  PRIMARY KEY CLUSTERED (component_name, cluster_id, service_name)
-  );
+  PRIMARY KEY CLUSTERED (id),
+  CONSTRAINT pk_servicecomponentdesiredstate PRIMARY KEY (id),
+  CONSTRAINT unq_scdesiredstate_name UNIQUE(component_name,service_name,cluster_id)
+);
+
+CREATE NONCLUSTERED INDEX idx_sc_desired_state ON servicecomponentdesiredstate(component_name, service_name, cluster_id);
 
 CREATE TABLE servicedesiredstate (
   cluster_id BIGINT NOT NULL,
@@ -1106,7 +1111,8 @@ BEGIN TRANSACTION
     ('topology_request_id_seq', 0),
     ('topology_host_group_id_seq', 0),
     ('setting_id_seq', 0),
-    ('hostcomponentstate_id_seq', 0);
+    ('hostcomponentstate_id_seq', 0),
+    ('servicecomponentdesiredstate_id_seq', 0);
 
   insert into adminresourcetype (resource_type_id, resource_type_name)
   values

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/test/java/org/apache/ambari/server/state/ServiceComponentTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/ServiceComponentTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/ServiceComponentTest.java
index ddab65d..4e8713b 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/ServiceComponentTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/ServiceComponentTest.java
@@ -39,7 +39,6 @@ import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
 import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -148,14 +147,8 @@ public class ServiceComponentTest {
     ServiceComponentDesiredStateDAO serviceComponentDesiredStateDAO =
         injector.getInstance(ServiceComponentDesiredStateDAO.class);
 
-    ServiceComponentDesiredStateEntityPK primaryKey =
-        new ServiceComponentDesiredStateEntityPK();
-    primaryKey.setClusterId(cluster.getClusterId());
-    primaryKey.setComponentName(componentName);
-    primaryKey.setServiceName(serviceName);
-
-    ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity =
-        serviceComponentDesiredStateDAO.findByPK(primaryKey);
+    ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity = serviceComponentDesiredStateDAO.findByName(
+        cluster.getClusterId(), serviceName, componentName);
 
     ServiceComponent sc1 = serviceComponentFactory.createExisting(service,
         serviceComponentDesiredStateEntity);

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog170Test.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog170Test.java b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog170Test.java
index 6bbcab7..10e4993 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog170Test.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog170Test.java
@@ -108,7 +108,6 @@ import org.apache.ambari.server.orm.entities.PrincipalEntity;
 import org.apache.ambari.server.orm.entities.PrivilegeEntity;
 import org.apache.ambari.server.orm.entities.ResourceEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.StackEntity;
 import org.apache.ambari.server.orm.entities.UserEntity;
 import org.apache.ambari.server.orm.entities.ViewEntity;
@@ -620,11 +619,9 @@ public class UpgradeCatalog170Test {
 
     upgradeCatalog170.moveHcatalogIntoHiveService();
 
-    ServiceComponentDesiredStateEntityPK pkHCATInHive = new ServiceComponentDesiredStateEntityPK();
-    pkHCATInHive.setComponentName("HCAT");
-    pkHCATInHive.setClusterId(clusterEntity.getClusterId());
-    pkHCATInHive.setServiceName("HIVE");
-    ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity = serviceComponentDesiredStateDAO.findByPK(pkHCATInHive);
+    ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity = serviceComponentDesiredStateDAO.findByName(
+        clusterEntity.getClusterId(), "HIVE", "HCAT");
+
     assertNotNull(serviceComponentDesiredStateEntity);
 
     HostComponentDesiredStateEntityPK hcDesiredStateEntityPk  = new HostComponentDesiredStateEntityPK();

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog200Test.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog200Test.java b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog200Test.java
index 8ff23f8..dcac986 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog200Test.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog200Test.java
@@ -71,7 +71,6 @@ import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.HostComponentStateEntity;
 import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.StackEntity;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
@@ -639,11 +638,9 @@ public class UpgradeCatalog200Test {
     upgradeCatalogHelper.addComponent(injector, clusterEntity,
         clusterServiceEntityNagios, hostEntity, "NAGIOS_SERVER", stackEntity);
 
-    ServiceComponentDesiredStateEntityPK pkNagiosServer = new ServiceComponentDesiredStateEntityPK();
-    pkNagiosServer.setComponentName("NAGIOS_SERVER");
-    pkNagiosServer.setClusterId(clusterEntity.getClusterId());
-    pkNagiosServer.setServiceName("NAGIOS");
-    ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity = serviceComponentDesiredStateDAO.findByPK(pkNagiosServer);
+    ServiceComponentDesiredStateEntity serviceComponentDesiredStateEntity = serviceComponentDesiredStateDAO.findByName(
+        clusterEntity.getClusterId(), "NAGIOS", "NAGIOS_SERVER");
+
     assertNotNull(serviceComponentDesiredStateEntity);
 
     HostComponentDesiredStateEntityPK hcDesiredStateEntityPk = new HostComponentDesiredStateEntityPK();

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog210Test.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog210Test.java b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog210Test.java
index 83018a2..c641bbf 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog210Test.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalog210Test.java
@@ -18,13 +18,37 @@
 
 package org.apache.ambari.server.upgrade;
 
-import com.google.inject.AbstractModule;
-import com.google.inject.Binder;
-import com.google.inject.Guice;
-import com.google.inject.Injector;
-import com.google.inject.Module;
-import com.google.inject.Provider;
-import com.google.inject.persist.PersistService;
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNotNull;
+import static junit.framework.Assert.assertNull;
+import static org.easymock.EasyMock.anyObject;
+import static org.easymock.EasyMock.capture;
+import static org.easymock.EasyMock.createMockBuilder;
+import static org.easymock.EasyMock.createNiceMock;
+import static org.easymock.EasyMock.createStrictMock;
+import static org.easymock.EasyMock.eq;
+import static org.easymock.EasyMock.expect;
+import static org.easymock.EasyMock.expectLastCall;
+import static org.easymock.EasyMock.replay;
+import static org.easymock.EasyMock.reset;
+import static org.easymock.EasyMock.verify;
+
+import java.io.File;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.net.URL;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+
+import javax.persistence.EntityManager;
+
 import org.apache.ambari.server.api.services.AmbariMetaInfo;
 import org.apache.ambari.server.configuration.Configuration;
 import org.apache.ambari.server.controller.AmbariManagementController;
@@ -46,7 +70,6 @@ import org.apache.ambari.server.orm.entities.ClusterStateEntity;
 import org.apache.ambari.server.orm.entities.HostComponentDesiredStateEntity;
 import org.apache.ambari.server.orm.entities.HostEntity;
 import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntity;
-import org.apache.ambari.server.orm.entities.ServiceComponentDesiredStateEntityPK;
 import org.apache.ambari.server.orm.entities.StackEntity;
 import org.apache.ambari.server.state.Cluster;
 import org.apache.ambari.server.state.Clusters;
@@ -66,35 +89,13 @@ import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
-import javax.persistence.EntityManager;
-import java.io.File;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.net.URL;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;
-import static org.easymock.EasyMock.anyObject;
-import static org.easymock.EasyMock.capture;
-import static org.easymock.EasyMock.createMockBuilder;
-import static org.easymock.EasyMock.createNiceMock;
-import static org.easymock.EasyMock.createStrictMock;
-import static org.easymock.EasyMock.eq;
-import static org.easymock.EasyMock.expect;
-import static org.easymock.EasyMock.expectLastCall;
-import static org.easymock.EasyMock.replay;
-import static org.easymock.EasyMock.reset;
-import static org.easymock.EasyMock.verify;
+import com.google.inject.AbstractModule;
+import com.google.inject.Binder;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import com.google.inject.Provider;
+import com.google.inject.persist.PersistService;
 
 /**
  * {@link org.apache.ambari.server.upgrade.UpgradeCatalog210} unit tests.
@@ -601,14 +602,15 @@ public class UpgradeCatalog210Test {
 
     Assert.assertEquals(HostComponentAdminState.INSERVICE.name(), entity.getAdminState().name());
 
+    // ensure the desired state exists
+    Assert.assertNotNull(componentDesiredStateDAO.findByName(clusterEntity.getClusterId(), "STORM",
+        "STORM_REST_API"));
+
     UpgradeCatalog210 upgradeCatalog210 = injector.getInstance(UpgradeCatalog210.class);
     upgradeCatalog210.removeStormRestApiServiceComponent();
 
-    ServiceComponentDesiredStateEntityPK entityPK = new ServiceComponentDesiredStateEntityPK();
-    entityPK.setClusterId(clusterEntity.getClusterId());
-    entityPK.setServiceName("STORM");
-    entityPK.setComponentName("STORM_REST_API");
-    Assert.assertNull(componentDesiredStateDAO.findByPK(entityPK));
+    Assert.assertNull(componentDesiredStateDAO.findByName(clusterEntity.getClusterId(), "STORM",
+        "STORM_REST_API"));
   }
 
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/697c309c/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalogHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalogHelper.java b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalogHelper.java
index 4c48972..4c11d10 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalogHelper.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/upgrade/UpgradeCatalogHelper.java
@@ -170,6 +170,9 @@ public class UpgradeCatalogHelper {
   protected void addComponent(Injector injector, ClusterEntity clusterEntity,
       ClusterServiceEntity clusterServiceEntity, HostEntity hostEntity,
       String componentName, StackEntity desiredStackEntity) {
+    ServiceComponentDesiredStateDAO serviceComponentDesiredStateDAO = injector.getInstance(
+        ServiceComponentDesiredStateDAO.class);
+
     ServiceComponentDesiredStateEntity componentDesiredStateEntity = new ServiceComponentDesiredStateEntity();
     componentDesiredStateEntity.setClusterServiceEntity(clusterServiceEntity);
     componentDesiredStateEntity.setComponentName(componentName);
@@ -177,6 +180,7 @@ public class UpgradeCatalogHelper {
     componentDesiredStateEntity.setDesiredStack(desiredStackEntity);
     componentDesiredStateEntity.setClusterServiceEntity(clusterServiceEntity);
     componentDesiredStateEntity.setClusterId(clusterServiceEntity.getClusterId());
+    serviceComponentDesiredStateDAO.create(componentDesiredStateEntity);
 
     HostComponentDesiredStateDAO hostComponentDesiredStateDAO = injector.getInstance(HostComponentDesiredStateDAO.class);
     HostComponentDesiredStateEntity hostComponentDesiredStateEntity = new HostComponentDesiredStateEntity();
@@ -208,7 +212,6 @@ public class UpgradeCatalogHelper {
         componentDesiredStateEntity);
 
     ClusterServiceDAO clusterServiceDAO = injector.getInstance(ClusterServiceDAO.class);
-    ServiceComponentDesiredStateDAO serviceComponentDesiredStateDAO = injector.getInstance(ServiceComponentDesiredStateDAO.class);
     HostDAO hostDAO = injector.getInstance(HostDAO.class);
     serviceComponentDesiredStateDAO.merge(componentDesiredStateEntity);
     hostDAO.merge(hostEntity);