You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@usergrid.apache.org by gr...@apache.org on 2015/06/10 18:45:27 UTC

[05/35] incubator-usergrid git commit: new search structure

new search structure


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

Branch: refs/heads/rest-test-framework-token-fix
Commit: 12ae65cc7c343a25d269cf1456ee7100567c9f68
Parents: 8a6e5a3
Author: Shawn Feldman <sf...@apache.org>
Authored: Fri Jun 5 10:41:08 2015 -0600
Committer: Shawn Feldman <sf...@apache.org>
Committed: Fri Jun 5 10:41:08 2015 -0600

----------------------------------------------------------------------
 .../persistence/index/EntityIndexFactory.java   |  5 +--
 .../impl/EsApplicationEntityIndexImpl.java      | 27 ++++++----------
 .../index/impl/EsEntityIndexFactoryImpl.java    | 29 +++++++----------
 .../index/impl/EsEntityIndexImpl.java           |  7 +++--
 .../index/impl/IndexLocationStrategy.java       | 33 ++++++++++++++++++++
 5 files changed, 62 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/12ae65cc/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexFactory.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexFactory.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexFactory.java
index ad2fba2..6490bb7 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexFactory.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/EntityIndexFactory.java
@@ -21,16 +21,17 @@ package org.apache.usergrid.persistence.index;
 
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import com.google.inject.assistedinject.Assisted;
+import org.apache.usergrid.persistence.index.impl.IndexLocationStrategy;
 
 
 public interface EntityIndexFactory {
 
     /**
      * Create an applicationEntity index for the provided scope
-     * @param appScope
+     * @param indexLocationStrategy
      * @return
      */
-    ApplicationEntityIndex createApplicationEntityIndex( ApplicationScope appScope );
+    ApplicationEntityIndex createApplicationEntityIndex( IndexLocationStrategy indexLocationStrategy );
 
     /**
      * Invalidate the cache of our factory, and force the generation of new entity index instances

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/12ae65cc/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsApplicationEntityIndexImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsApplicationEntityIndexImpl.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsApplicationEntityIndexImpl.java
index ed35838..74bb33c 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsApplicationEntityIndexImpl.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsApplicationEntityIndexImpl.java
@@ -80,11 +80,8 @@ public class EsApplicationEntityIndexImpl implements ApplicationEntityIndex {
     private final ApplicationScope applicationScope;
     private final IndexIdentifier indexIdentifier;
     private final Timer searchTimer;
-    private final Timer cursorTimer;
-    private final MapManager mapManager;
     private final AliasedEntityIndex entityIndex;
     private final IndexBufferConsumer indexBatchBufferProducer;
-    private final IndexFig indexFig;
     private final EsProvider esProvider;
     private final IndexAlias alias;
     private final Timer deleteApplicationTimer;
@@ -97,25 +94,20 @@ public class EsApplicationEntityIndexImpl implements ApplicationEntityIndex {
 
 
     @Inject
-    public EsApplicationEntityIndexImpl(  ApplicationScope appScope, final AliasedEntityIndex entityIndex,
-                                         final IndexFig config, final IndexBufferConsumer indexBatchBufferProducer,
+    public EsApplicationEntityIndexImpl(  final AliasedEntityIndex entityIndex,
+                                         final IndexFig config,
+                                          final IndexBufferConsumer indexBatchBufferProducer,
                                          final EsProvider provider,
-                                         final MetricsFactory metricsFactory, final MapManagerFactory mapManagerFactory,
-                                         final IndexFig indexFig,
-                                         final IndexIdentifier indexIdentifier ) {
+                                         final MetricsFactory metricsFactory,
+                                         final IndexLocationStrategy indexLocationStrategy
+    ) {
         this.entityIndex = entityIndex;
         this.indexBatchBufferProducer = indexBatchBufferProducer;
-        this.indexFig = indexFig;
-        this.indexIdentifier = indexIdentifier;
-        ValidationUtils.validateApplicationScope( appScope );
-        this.applicationScope = appScope;
-        final MapScope mapScope = new MapScopeImpl( appScope.getApplication(), "cursorcache" );
+        this.indexIdentifier = indexLocationStrategy.getIndexIdentifier();
         this.failureMonitor = new FailureMonitorImpl( config, provider );
         this.esProvider = provider;
 
-        mapManager = mapManagerFactory.createMapManager( mapScope );
         this.searchTimer = metricsFactory.getTimer( EsApplicationEntityIndexImpl.class, "search.timer" );
-        this.cursorTimer = metricsFactory.getTimer( EsApplicationEntityIndexImpl.class, "search.cursor.timer" );
         this.cursorTimeout = config.getQueryCursorTimeout();
         this.queryTimeout = config.getWriteTimeout();
 
@@ -126,8 +118,9 @@ public class EsApplicationEntityIndexImpl implements ApplicationEntityIndex {
 
         this.alias = indexIdentifier.getAlias();
 
-        this.searchRequest = new SearchRequestBuilderStrategy( esProvider, appScope, alias, cursorTimeout );
-        this.searchRequestBuilderStrategyV2 = new SearchRequestBuilderStrategyV2( esProvider, appScope, alias, cursorTimeout  );
+        this.applicationScope = indexLocationStrategy.getApplicationScope();
+        this.searchRequest = new SearchRequestBuilderStrategy( esProvider, applicationScope, alias, cursorTimeout );
+        this.searchRequestBuilderStrategyV2 = new SearchRequestBuilderStrategyV2( esProvider, applicationScope, alias, cursorTimeout  );
     }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/12ae65cc/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexFactoryImpl.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexFactoryImpl.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexFactoryImpl.java
index b32fec4..4b1d062 100644
--- a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexFactoryImpl.java
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/EsEntityIndexFactoryImpl.java
@@ -25,7 +25,6 @@ import java.util.concurrent.ExecutionException;
 import org.apache.usergrid.persistence.core.metrics.MetricsFactory;
 import org.apache.usergrid.persistence.core.scope.ApplicationScope;
 import org.apache.usergrid.persistence.index.*;
-import org.apache.usergrid.persistence.map.MapManagerFactory;
 
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
@@ -41,41 +40,37 @@ public class EsEntityIndexFactoryImpl implements EntityIndexFactory{
     private final EsProvider provider;
     private final IndexBufferConsumer indexBatchBufferProducer;
     private final MetricsFactory metricsFactory;
-    private final MapManagerFactory mapManagerFactory;
-    private final IndexFig indexFig;
     private final AliasedEntityIndex entityIndex;
-    private final IndexIdentifier indexIdentifier;
 
-    private LoadingCache<ApplicationScope, ApplicationEntityIndex> eiCache =
-        CacheBuilder.newBuilder().maximumSize( 1000 ).build( new CacheLoader<ApplicationScope, ApplicationEntityIndex>() {
-            public ApplicationEntityIndex load( ApplicationScope scope ) {
+    private LoadingCache<IndexLocationStrategy, ApplicationEntityIndex> eiCache =
+        CacheBuilder.newBuilder().maximumSize( 1000 ).build( new CacheLoader<IndexLocationStrategy, ApplicationEntityIndex>() {
+            public ApplicationEntityIndex load( IndexLocationStrategy locationStrategy ) {
                 return new EsApplicationEntityIndexImpl(
-                    scope,entityIndex,config, indexBatchBufferProducer, provider, metricsFactory, mapManagerFactory, indexFig, indexIdentifier
+                    entityIndex,config, indexBatchBufferProducer, provider, metricsFactory, locationStrategy
                 );
             }
         } );
 
     @Inject
-    public EsEntityIndexFactoryImpl( final IndexFig config, final EsProvider provider,
+    public EsEntityIndexFactoryImpl( final IndexFig indexFig, final EsProvider provider,
                                      final IndexBufferConsumer indexBatchBufferProducer,
-                                     final MetricsFactory metricsFactory, final MapManagerFactory mapManagerFactory,
-                                     final IndexFig indexFig, final AliasedEntityIndex entityIndex, final IndexIdentifier indexIdentifier ){
-        this.config = config;
+                                     final MetricsFactory metricsFactory,
+                                     final AliasedEntityIndex entityIndex
+
+    ){
+        this.config = indexFig;
         this.provider = provider;
         this.indexBatchBufferProducer = indexBatchBufferProducer;
         this.metricsFactory = metricsFactory;
-        this.mapManagerFactory = mapManagerFactory;
-        this.indexFig = indexFig;
         this.entityIndex = entityIndex;
-        this.indexIdentifier = indexIdentifier;
     }
 
 
 
     @Override
-    public ApplicationEntityIndex createApplicationEntityIndex(final ApplicationScope appScope) {
+    public ApplicationEntityIndex createApplicationEntityIndex(final IndexLocationStrategy indexLocationStrategy) {
         try{
-            return eiCache.get(appScope);
+            return eiCache.get(indexLocationStrategy);
         }catch (ExecutionException ee){
             throw new RuntimeException(ee);
         }

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/12ae65cc/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 7fb3b85..5a4878f 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
@@ -26,6 +26,7 @@ import com.google.common.io.Resources;
 import com.google.inject.Inject;
 import com.google.inject.Singleton;
 
+import com.google.inject.assistedinject.Assisted;
 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;
@@ -94,13 +95,13 @@ public class EsEntityIndexImpl implements AliasedEntityIndex,VersionedData {
     public EsEntityIndexImpl( final EsProvider provider,
                               final IndexCache indexCache,
                               final IndexFig indexFig,
-                              final IndexIdentifier indexIdentifier,
                               final IndexRefreshCommand indexRefreshCommand,
-                              final MetricsFactory metricsFactory
+                              final MetricsFactory metricsFactory,
+                              @Assisted final IndexLocationStrategy indexLocationStrategy
     ) {
 
         this.indexFig = indexFig;
-        this.indexIdentifier = indexIdentifier;
+        this.indexIdentifier = indexLocationStrategy.getIndexIdentifier();
 
         this.esProvider = provider;
         this.indexRefreshCommand = indexRefreshCommand;

http://git-wip-us.apache.org/repos/asf/incubator-usergrid/blob/12ae65cc/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/IndexLocationStrategy.java
----------------------------------------------------------------------
diff --git a/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/IndexLocationStrategy.java b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/IndexLocationStrategy.java
new file mode 100644
index 0000000..89ecdeb
--- /dev/null
+++ b/stack/corepersistence/queryindex/src/main/java/org/apache/usergrid/persistence/index/impl/IndexLocationStrategy.java
@@ -0,0 +1,33 @@
+/*
+ *
+ *  * 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.impl;
+
+import org.apache.usergrid.persistence.core.scope.ApplicationScope;
+import org.apache.usergrid.persistence.index.IndexIdentifier;
+
+/**
+ * Classy class class.
+ */
+public interface IndexLocationStrategy {
+    IndexIdentifier getIndexIdentifier();
+    ApplicationScope getApplicationScope();
+    void validateApplicationScope();//        ValidationUtils.validateApplicationScope( appScope );
+
+}