You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@usergrid.apache.org by sf...@apache.org on 2015/03/31 23:01:05 UTC

[27/50] incubator-usergrid git commit: add version migration for indexes

add version migration for indexes


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

Branch: refs/heads/USERGRID-473
Commit: bf0718e611bb06d8bf901ba34f5d8172d5ef02bb
Parents: 208ed2b
Author: Shawn Feldman <sf...@apache.org>
Authored: Thu Mar 26 10:53:20 2015 -0600
Committer: Shawn Feldman <sf...@apache.org>
Committed: Thu Mar 26 10:53:20 2015 -0600

----------------------------------------------------------------------
 .../AllApplicationsObservable.java              |  3 +-
 .../usergrid/corepersistence/CoreModule.java    |  9 ++++-
 .../rx/impl/AllApplicationsObservableImpl.java  |  5 ++-
 .../persistence/index/guice/IndexModule.java    | 13 ++++++-
 .../index/impl/EsEntityIndexImpl.java           |  9 ++++-
 .../migration/EsIndexDataMigrationImpl.java     | 20 ++++++++--
 .../index/migration/IndexDataVersions.java      | 39 ++++++++++++++++++++
 .../index/guice/TestIndexModule.java            |  7 +++-
 8 files changed, 94 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/bf0718e6/stack/core/src/main/java/org/apache/usergrid/corepersistence/AllApplicationsObservable.java
----------------------------------------------------------------------
diff --git a/stack/core/src/main/java/org/apache/usergrid/corepersistence/AllApplicationsObservable.java b/stack/core/src/main/java/org/apache/usergrid/corepersistence/AllApplicationsObservable.java
index 1187cf2..055223e 100644
--- a/stack/core/src/main/java/org/apache/usergrid/corepersistence/AllApplicationsObservable.java
+++ b/stack/core/src/main/java/org/apache/usergrid/corepersistence/AllApplicationsObservable.java
@@ -20,6 +20,7 @@
 package org.apache.usergrid.corepersistence;
 
 
+import org.apache.usergrid.persistence.core.migration.data.MigrationDataProvider;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.entities.Application;
 
@@ -29,7 +30,7 @@ import rx.Observable;
 /**
  * Interface for generating an observable of all ApplicationScope
  */
-public interface AllApplicationsObservable {
+public interface AllApplicationsObservable extends MigrationDataProvider<ApplicationScope>{
 
     /**
      * Return all applications in our system

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/bf0718e6/stack/core/src/main/java/org/apache/usergrid/corepersistence/CoreModule.java
----------------------------------------------------------------------
diff --git a/stack/core/src/main/java/org/apache/usergrid/corepersistence/CoreModule.java b/stack/core/src/main/java/org/apache/usergrid/corepersistence/CoreModule.java
index 2e9b780..d994419 100644
--- a/stack/core/src/main/java/org/apache/usergrid/corepersistence/CoreModule.java
+++ b/stack/core/src/main/java/org/apache/usergrid/corepersistence/CoreModule.java
@@ -16,6 +16,7 @@
 package org.apache.usergrid.corepersistence;
 
 
+import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.springframework.context.ApplicationContext;
 
 import org.apache.usergrid.corepersistence.events.EntityDeletedHandler;
@@ -96,7 +97,13 @@ public class CoreModule  extends AbstractModule {
                     AllNodesInGraphImpl.class );
             }
         } );
-        install(new IndexModule());
+        install(new IndexModule(){
+            @Override
+            public void configureMigrationProvider() {
+                bind( new TypeLiteral<MigrationDataProvider<ApplicationScope>>() {} ).to(
+                    AllApplicationsObservable.class );
+            }
+        });
        //        install(new MapModule());   TODO, re-enable when index module doesn't depend on queue
        //        install(new QueueModule());
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/bf0718e6/stack/core/src/main/java/org/apache/usergrid/corepersistence/rx/impl/AllApplicationsObservableImpl.java
----------------------------------------------------------------------
diff --git a/stack/core/src/main/java/org/apache/usergrid/corepersistence/rx/impl/AllApplicationsObservableImpl.java b/stack/core/src/main/java/org/apache/usergrid/corepersistence/rx/impl/AllApplicationsObservableImpl.java
index 0fc5452..aa5b5dc 100644
--- a/stack/core/src/main/java/org/apache/usergrid/corepersistence/rx/impl/AllApplicationsObservableImpl.java
+++ b/stack/core/src/main/java/org/apache/usergrid/corepersistence/rx/impl/AllApplicationsObservableImpl.java
@@ -138,5 +138,8 @@ public class AllApplicationsObservableImpl implements AllApplicationsObservable
     }
 
 
-
+    @Override
+    public Observable<ApplicationScope> getData() {
+        return getAllApplications();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/bf0718e6/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/guice/IndexModule.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/guice/IndexModule.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/guice/IndexModule.java
index a42dea8..7ecce54 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/guice/IndexModule.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/guice/IndexModule.java
@@ -42,7 +42,7 @@ import org.apache.usergrid.persistence.queue.guice.QueueModule;
 import org.safehaus.guicyfig.GuicyFigModule;
 
 
-public class IndexModule extends AbstractModule {
+public abstract class IndexModule extends AbstractModule {
 
     @Override
     protected void configure() {
@@ -75,8 +75,17 @@ public class IndexModule extends AbstractModule {
 
         //wire up the collection migration plugin
         Multibinder.newSetBinder( binder(), MigrationPlugin.class ).addBinding().to(EsIndexMigrationPlugin.class);
-    }
 
 
+        //invoke the migration plugin config
+        configureMigrationProvider();
+    }
+
+    /**
+     * Gives callers the ability to to configure an instance of
+     *
+     * MigrationDataProvider<ApplicationScope> for providing data migrations
+     */
+    public abstract void configureMigrationProvider();
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/bf0718e6/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexImpl.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexImpl.java
index cab8ded..844260f 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexImpl.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexImpl.java
@@ -27,10 +27,12 @@ import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.usergrid.persistence.core.future.BetterFuture;
 import org.apache.usergrid.persistence.core.metrics.MetricsFactory;
+import org.apache.usergrid.persistence.core.migration.data.VersionedData;
 import org.apache.usergrid.persistence.core.util.Health;
 import org.apache.usergrid.persistence.index.*;
 import org.apache.usergrid.persistence.index.exceptions.IndexException;
 
+import org.apache.usergrid.persistence.index.migration.IndexDataVersions;
 import org.apache.usergrid.persistence.model.util.UUIDGenerator;
 
 import org.elasticsearch.action.ActionFuture;
@@ -71,7 +73,7 @@ import java.util.*;
  * Implements index using ElasticSearch Java API.
  */
 @Singleton
-public class EsEntityIndexImpl implements AliasedEntityIndex {
+public class EsEntityIndexImpl implements AliasedEntityIndex,VersionedData {
 
     private static final Logger logger = LoggerFactory.getLogger( EsEntityIndexImpl.class );
 
@@ -442,6 +444,11 @@ public class EsEntityIndexImpl implements AliasedEntityIndex {
         return Health.RED;
     }
 
+    @Override
+    public int getImplementationVersion() {
+        return IndexDataVersions.SINGLE_INDEX.getVersion();
+    }
+
 
     /**
      * Interface for operations.

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/bf0718e6/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/EsIndexDataMigrationImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/EsIndexDataMigrationImpl.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/EsIndexDataMigrationImpl.java
index b5dab53..2a0bb15 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/EsIndexDataMigrationImpl.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/EsIndexDataMigrationImpl.java
@@ -20,6 +20,7 @@ import com.google.inject.Inject;
 import org.apache.usergrid.persistence.core.migration.data.DataMigration;
 import org.apache.usergrid.persistence.core.migration.data.MigrationDataProvider;
 import org.apache.usergrid.persistence.core.migration.data.ProgressObserver;
+import org.apache.usergrid.persistence.core.migration.data.VersionedData;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.index.AliasedEntityIndex;
 import org.apache.usergrid.persistence.index.IndexAlias;
@@ -30,9 +31,12 @@ import org.apache.usergrid.persistence.index.impl.EsProvider;
 import org.apache.usergrid.persistence.index.impl.IndexingUtils;
 import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequestBuilder;
 import org.elasticsearch.client.AdminClient;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
 
 /**
  * Classy class class.
@@ -44,6 +48,8 @@ public class EsIndexDataMigrationImpl implements DataMigration<ApplicationScope>
     private final IndexFig indexFig;
     private final IndexIdentifier indexIdentifier;
     private final EsIndexCache indexCache;
+    private final VersionedData dataVersion;
+    private static final Logger log = LoggerFactory.getLogger(EsIndexDataMigrationImpl.class);
 
     @Inject
     public EsIndexDataMigrationImpl(AliasedEntityIndex entityIndex, EsProvider provider, IndexFig indexFig, IndexIdentifier indexIdentifier, EsIndexCache indexCache){
@@ -52,10 +58,12 @@ public class EsIndexDataMigrationImpl implements DataMigration<ApplicationScope>
         this.indexFig = indexFig;
         this.indexIdentifier = indexIdentifier;
         this.indexCache = indexCache;
+        this.dataVersion = (VersionedData) entityIndex;
     }
 
     @Override
     public int migrate(int currentVersion, MigrationDataProvider<ApplicationScope> migrationDataProvider, ProgressObserver observer) {
+        final AtomicInteger integer = new AtomicInteger();
         migrationDataProvider.getData().doOnNext(applicationScope -> {
             LegacyIndexIdentifier legacyIndexIdentifier = new LegacyIndexIdentifier(indexFig,applicationScope);
             String[] indexes = indexCache.getIndexes(legacyIndexIdentifier.getAlias(), AliasedEntityIndex.AliasType.Read);
@@ -66,19 +74,23 @@ public class EsIndexDataMigrationImpl implements DataMigration<ApplicationScope>
                 aliasesRequestBuilder = adminClient.indices().prepareAliases();
                 // add read alias
                 aliasesRequestBuilder.addAlias(index, indexIdentifier.getAlias().getReadAlias());
+                integer.incrementAndGet();
             }
-        });
-        return 0;
+        })
+        .doOnError(error -> log.error("failed to migrate index",error))
+        .toBlocking().last();
+
+        return integer.get();
     }
 
     @Override
     public boolean supports(int currentVersion) {
-        return false;
+        return currentVersion < dataVersion.getImplementationVersion();
     }
 
     @Override
     public int getMaxVersion() {
-        return 0;
+        return dataVersion.getImplementationVersion();
     }
     /**
      * Class is used to generate an index name and alias name the old way via app name

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/bf0718e6/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/IndexDataVersions.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/IndexDataVersions.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/IndexDataVersions.java
new file mode 100644
index 0000000..921d86a
--- /dev/null
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/migration/IndexDataVersions.java
@@ -0,0 +1,39 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one or more
+ *  *  contributor license agreements.  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.  For additional information regarding
+ *  * copyright in this work, please see the NOTICE file in the top level
+ *  * directory of this distribution.
+ *
+ */
+
+package org.apache.usergrid.persistence.index.migration;
+
+/**
+ * Classy class class.
+ */
+public enum IndexDataVersions {
+    MANY_INDEXES(0),
+    SINGLE_INDEX(1);
+
+    private final int version;
+
+
+    private IndexDataVersions( final int version ) {this.version = version;}
+
+
+    public int getVersion() {
+        return version;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/bf0718e6/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/guice/TestIndexModule.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/guice/TestIndexModule.java b/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/guice/TestIndexModule.java
index 4cf46d6..4e6700c 100644
--- a/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/guice/TestIndexModule.java
+++ b/stack/corepersistence/queryindex/src/test/java/org/apache/usergrid/persistence/index/guice/TestIndexModule.java
@@ -33,7 +33,12 @@ public class TestIndexModule extends TestModule {
         install( new CommonModule());
 
         // configure collections and our core astyanax framework
-        install( new IndexModule()  );
+        install( new IndexModule(){
+            @Override
+            public  void configureMigrationProvider(){
+
+            }
+        });
         install( new GuicyFigModule(IndexTestFig.class) );
     }
 }