You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@usergrid.apache.org by sn...@apache.org on 2014/10/24 16:48:18 UTC

git commit: Implemented deleteEntity() from all index scopes methods, using ES delete by query, next up: tests.

Repository: incubator-usergrid
Updated Branches:
  refs/heads/two-dot-o-events 9fc4f0afa -> 5f28208f5


Implemented deleteEntity() from all index scopes methods, using ES delete by query, next up: tests.


Project: http://git-wip-us.apache.org/repos/asf/incubator-usergrid/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-usergrid/commit/5f28208f
Tree: http://git-wip-us.apache.org/repos/asf/incubator-usergrid/tree/5f28208f
Diff: http://git-wip-us.apache.org/repos/asf/incubator-usergrid/diff/5f28208f

Branch: refs/heads/two-dot-o-events
Commit: 5f28208f5fc018646fd4376216c172c792fc6d6d
Parents: 9fc4f0a
Author: Dave Johnson <dm...@apigee.com>
Authored: Fri Oct 24 10:47:55 2014 -0400
Committer: Dave Johnson <dm...@apigee.com>
Committed: Fri Oct 24 10:47:55 2014 -0400

----------------------------------------------------------------------
 .../events/EntityDeletedImpl.java               | 85 ++------------------
 .../usergrid/persistence/index/EntityIndex.java |  7 +-
 .../persistence/index/EntityIndexBatch.java     | 10 +--
 .../index/impl/EsEntityIndexBatchImpl.java      | 37 +++++++--
 .../persistence/index/impl/EsProvider.java      |  3 -
 5 files changed, 41 insertions(+), 101 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5f28208f/stack/core/src/main/java/org/apache/usergrid/corepersistence/events/EntityDeletedImpl.java
----------------------------------------------------------------------
diff --git a/stack/core/src/main/java/org/apache/usergrid/corepersistence/events/EntityDeletedImpl.java b/stack/core/src/main/java/org/apache/usergrid/corepersistence/events/EntityDeletedImpl.java
index bb30c8c..0c49e84 100644
--- a/stack/core/src/main/java/org/apache/usergrid/corepersistence/events/EntityDeletedImpl.java
+++ b/stack/core/src/main/java/org/apache/usergrid/corepersistence/events/EntityDeletedImpl.java
@@ -18,25 +18,15 @@
 
 package org.apache.usergrid.corepersistence.events;
 
-import java.util.Map;
-import java.util.Set;
 import org.apache.usergrid.persistence.collection.CollectionScope;
 import org.apache.usergrid.persistence.collection.event.EntityDeleted;
 import org.apache.usergrid.persistence.index.EntityIndexBatch;
-import org.apache.usergrid.persistence.index.IndexScope;
-import org.apache.usergrid.persistence.index.impl.IndexScopeImpl;
 import org.apache.usergrid.persistence.model.entity.Id;
-import org.apache.usergrid.persistence.model.entity.SimpleId;
 
 import java.util.UUID;
-import org.apache.usergrid.corepersistence.CpEntityManager;
 import org.apache.usergrid.corepersistence.CpEntityManagerFactory;
 import org.apache.usergrid.corepersistence.CpSetup;
-import org.apache.usergrid.corepersistence.util.CpNamingUtils;
 import org.apache.usergrid.persistence.EntityManagerFactory;
-import org.apache.usergrid.persistence.RelationManager;
-import org.apache.usergrid.persistence.SimpleEntityRef;
-import org.apache.usergrid.persistence.collection.EntityCollectionManager;
 import org.apache.usergrid.persistence.index.EntityIndex;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -49,10 +39,6 @@ public class EntityDeletedImpl implements EntityDeleted {
     private static final Logger logger = LoggerFactory.getLogger( EntityDeletedImpl.class );
 
 
-    public EntityDeletedImpl(){
-
-    }
-
     @Override
     public void deleted(CollectionScope scope, Id entityId, UUID version) {
 
@@ -61,75 +47,14 @@ public class EntityDeletedImpl implements EntityDeleted {
             new Object[] { entityId.getType(), entityId.getUuid(), version,
                 scope.getName(), scope.getOwner(), scope.getApplication()});
 
-
         CpEntityManagerFactory emf = (CpEntityManagerFactory)
-                CpSetup.getInjector().getInstance( EntityManagerFactory.class );
-
-        CpEntityManager em = (CpEntityManager)
-                emf.getEntityManager( scope.getOwner().getUuid() );
-
-        EntityCollectionManager ecm = emf.getManagerCache().getEntityCollectionManager(scope);
-
-
-        // TODO: change this so that it gets every version of the entity that 
-        // exists as we need to de-index each and every one of them
-
-        org.apache.usergrid.persistence.model.entity.Entity entity = 
-            ecm.load( entityId ).toBlocking().last();
-
-
-        SimpleEntityRef entityRef = new SimpleEntityRef( entityId.getType(), entityId.getUuid());
-
-        if ( entity != null ) {
-
-            // first, delete entity in every collection and connection scope in which it is indexed 
-
-            RelationManager rm = em.getRelationManager( entityRef );
-            Map<String, Map<UUID, Set<String>>> owners = null;
-            try {
-                owners = rm.getOwners();
-
-                logger.debug( "Deleting indexes of all {} collections owning the entity {}:{}", 
-                    new Object[] { owners.keySet().size(), entityId.getType(), entityId.getUuid()});
-
-                final EntityIndex ei = emf.getManagerCache().getEntityIndex(scope);
-
-                final EntityIndexBatch batch = ei.createBatch();
-
-                for ( String ownerType : owners.keySet() ) {
-                    Map<UUID, Set<String>> collectionsByUuid = owners.get( ownerType );
-
-                    for ( UUID uuid : collectionsByUuid.keySet() ) {
-                        Set<String> collectionNames = collectionsByUuid.get( uuid );
-                        for ( String coll : collectionNames ) {
-
-                            IndexScope indexScope = new IndexScopeImpl(
-                                new SimpleId( uuid, ownerType ), 
-                                CpNamingUtils.getCollectionScopeNameFromCollectionName( coll ));
-
-                            batch.index( indexScope, entity );
-                        }
-                    }
-                }
-
-                // deindex from default index scope
-                IndexScope defaultIndexScope = new IndexScopeImpl( scope.getApplication(),
-                    CpNamingUtils.getCollectionScopeNameFromEntityType( entityRef.getType()));
-
-                batch.deindex(defaultIndexScope,  entity );
-
-                IndexScope allTypesIndexScope = new IndexScopeImpl(
-                    scope.getApplication(), CpNamingUtils.ALL_TYPES);
+            CpSetup.getInjector().getInstance( EntityManagerFactory.class );
 
-                batch.deindex( allTypesIndexScope,  entity );
+        final EntityIndex ei = emf.getManagerCache().getEntityIndex(scope);
 
-                batch.execute();
+        EntityIndexBatch batch = ei.createBatch();
 
-            } catch (Exception e) {
-                logger.error("Cannot deindex from owners of entity {}:{}", 
-                        entityId.getType(), entityId.getUuid());
-                logger.error("The exception", e);
-            }
-        }
+        batch.deleteEntity( entityId );
+        batch.execute();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5f28208f/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndex.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndex.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndex.java
index 44ad05c..6c4529c 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndex.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndex.java
@@ -30,14 +30,13 @@ import org.apache.usergrid.persistence.model.entity.Id;
 public interface EntityIndex {
 
     /**
-     * This should ONLY ever be called once on application create.  Otherwise we're introducing slowness into our system
-     *
+     * This should ONLY ever be called once on application create.  
+     * Otherwise we're introducing slowness into our system.
      */
     public void initializeIndex();
 
     /**
      * Create the index batch
-     * @return
      */
     public EntityIndexBatch createBatch();
 
@@ -49,8 +48,6 @@ public interface EntityIndex {
 
     /**
      * Get the candidate results of all versions of the entity for this id
-     * @param id
-     * @return
      */
     public CandidateResults getEntityVersions(final IndexScope indexScope, Id id);
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5f28208f/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexBatch.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexBatch.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexBatch.java
index f98025b..04992e1 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexBatch.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexBatch.java
@@ -27,9 +27,6 @@ import org.apache.usergrid.persistence.model.entity.Id;
 
 public interface EntityIndexBatch {
 
-
-
-
     /**
      * Create index for Entity
      * @param indexScope The scope for the index
@@ -59,8 +56,11 @@ public interface EntityIndexBatch {
      */
     public EntityIndexBatch deindex(final IndexScope scope, final Id id, final UUID version);
 
-
-    //TODO: Create a delete method that delete's  by Id.  This will delete all documents from ES with the same entity Id
+    /**
+     * Create a delete method that deletes by Id. This will delete all documents from ES with 
+     * the same entity Id, effectively removing all versions of an entity from all index scopes.
+     */
+    public EntityIndexBatch deleteEntity( Id entityId ); 
 
     /**
      * Execute the batch

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5f28208f/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexBatchImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexBatchImpl.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexBatchImpl.java
index 151e850..8858008 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexBatchImpl.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexBatchImpl.java
@@ -68,11 +68,14 @@ import static org.apache.usergrid.persistence.index.impl.IndexingUtils.STRING_PR
 import static org.apache.usergrid.persistence.index.impl.IndexingUtils.createCollectionScopeTypeName;
 import static org.apache.usergrid.persistence.index.impl.IndexingUtils.createIndexDocId;
 import static org.apache.usergrid.persistence.index.impl.IndexingUtils.createIndexName;
+import org.elasticsearch.action.deletebyquery.DeleteByQueryResponse;
+import org.elasticsearch.index.query.QueryBuilders;
+import org.elasticsearch.index.query.TermQueryBuilder;
 
 
 public class EsEntityIndexBatchImpl implements EntityIndexBatch {
 
-    private static final Logger log = LoggerFactory.getLogger( EsEntityIndexBatchImpl.class );
+    private static final Logger logger = LoggerFactory.getLogger( EsEntityIndexBatchImpl.class );
 
     private final ApplicationScope applicationScope;
 
@@ -107,13 +110,12 @@ public class EsEntityIndexBatchImpl implements EntityIndexBatch {
     @Override
     public EntityIndexBatch index( final IndexScope indexScope, final Entity entity ) {
 
-
         IndexValidationUtils.validateIndexScope( indexScope );
 
         final String indexType = createCollectionScopeTypeName( indexScope );
 
-        if ( log.isDebugEnabled() ) {
-            log.debug( "Indexing entity {}:{} in scope\n   app {}\n   "
+        if ( logger.isDebugEnabled() ) {
+            logger.debug( "Indexing entity {}:{} in scope\n   app {}\n   "
                 + "owner {}\n   name {}\n   type {}", new Object[] {
                     entity.getId().getType(), 
                     entity.getId().getUuid(), 
@@ -137,7 +139,7 @@ public class EsEntityIndexBatchImpl implements EntityIndexBatch {
 
         String indexId = createIndexDocId( entity );
 
-        log.debug( "Indexing entity id {} data {} ", indexId, entityAsMap );
+        logger.debug( "Indexing entity id {} data {} ", indexId, entityAsMap );
 
         bulkRequest.add(client.prepareIndex( indexName, indexType, indexId).setSource(entityAsMap));
 
@@ -154,8 +156,8 @@ public class EsEntityIndexBatchImpl implements EntityIndexBatch {
 
         final String indexType = createCollectionScopeTypeName( indexScope );
 
-        if ( log.isDebugEnabled() ) {
-            log.debug( "De-indexing entity {}:{} in scope\n   app {}\n   owner {}\n   "
+        if ( logger.isDebugEnabled() ) {
+            logger.debug( "De-indexing entity {}:{} in scope\n   app {}\n   owner {}\n   "
                 + "name {} type {}", new Object[] {
                     id.getType(), 
                     id.getUuid(), 
@@ -170,7 +172,7 @@ public class EsEntityIndexBatchImpl implements EntityIndexBatch {
 
         bulkRequest.add( client.prepareDelete( indexName, indexType, indexId ).setRefresh(refresh));
 
-        log.debug( "Deindexed Entity with index id " + indexId );
+        logger.debug( "Deindexed Entity with index id " + indexId );
 
         maybeFlush();
 
@@ -191,6 +193,24 @@ public class EsEntityIndexBatchImpl implements EntityIndexBatch {
         return deindex( indexScope, entity.getId(), entity.getVersion() );
     }
 
+    
+    @Override
+    public EntityIndexBatch deleteEntity(Id entityId) {
+
+        TermQueryBuilder tqb = QueryBuilders.termQuery(
+            STRING_PREFIX + ENTITYID_FIELDNAME, entityId.getUuid().toString().toLowerCase());
+
+        DeleteByQueryResponse response = client.prepareDeleteByQuery("test")
+            .setQuery( tqb ).execute().actionGet();
+
+        logger.debug("Deleted entity {}:{} from all index scopes with response status = {}", 
+            new Object[] { entityId.getType(), entityId.getUuid(), response.status().toString() });
+
+        maybeFlush();
+
+        return this;
+    }
+
 
     @Override
     public void execute() {
@@ -360,4 +380,5 @@ public class EsEntityIndexBatchImpl implements EntityIndexBatch {
     private void initBatch() {
         this.bulkRequest = client.prepareBulk();
     }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/5f28208f/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsProvider.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsProvider.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsProvider.java
index 9b37952..adec21c 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsProvider.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsProvider.java
@@ -25,16 +25,13 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Properties;
-import java.util.logging.Level;
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.lang.RandomStringUtils;
 import org.apache.usergrid.persistence.core.util.AvailablePortFinder;
 import org.apache.usergrid.persistence.index.IndexFig;
 import org.elasticsearch.client.Client;
-import org.elasticsearch.client.transport.TransportClient;
 import org.elasticsearch.common.settings.ImmutableSettings;
 import org.elasticsearch.common.settings.Settings;
-import org.elasticsearch.common.transport.InetSocketTransportAddress;
 import org.elasticsearch.node.Node;
 import org.elasticsearch.node.NodeBuilder;
 import org.slf4j.Logger;