You are viewing a plain text version of this content. The canonical link for it is here.
Posted to oak-commits@jackrabbit.apache.org by to...@apache.org on 2014/12/18 15:40:54 UTC

svn commit: r1646469 - in /jackrabbit/oak/trunk/oak-solr-core/src: main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/ main/java/org/apache/jackrabbit/oak/plugins/index/solr/osgi/ main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/ ...

Author: tommaso
Date: Thu Dec 18 14:40:53 2014
New Revision: 1646469

URL: http://svn.apache.org/r1646469
Log:
OAK-2168 - support qt aggregation (wip)

Modified:
    jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexEditorProvider.java
    jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/osgi/SolrQueryIndexProviderService.java
    jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndex.java
    jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexProvider.java
    jackrabbit/oak/trunk/oak-solr-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexTest.java

Modified: jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexEditorProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexEditorProvider.java?rev=1646469&r1=1646468&r2=1646469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexEditorProvider.java (original)
+++ jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/SolrIndexEditorProvider.java Thu Dec 18 14:40:53 2014
@@ -16,6 +16,8 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.solr.index;
 
+import javax.annotation.Nonnull;
+
 import org.apache.jackrabbit.oak.api.CommitFailedException;
 import org.apache.jackrabbit.oak.plugins.index.IndexEditorProvider;
 import org.apache.jackrabbit.oak.plugins.index.IndexUpdateCallback;
@@ -29,8 +31,6 @@ import org.apache.solr.client.solrj.Solr
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.annotation.Nonnull;
-
 /**
  * Solr based {@link IndexEditorProvider}
  *

Modified: jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/osgi/SolrQueryIndexProviderService.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/osgi/SolrQueryIndexProviderService.java?rev=1646469&r1=1646468&r2=1646469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/osgi/SolrQueryIndexProviderService.java (original)
+++ jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/osgi/SolrQueryIndexProviderService.java Thu Dec 18 14:40:53 2014
@@ -19,15 +19,27 @@ package org.apache.jackrabbit.oak.plugin
 import java.util.ArrayList;
 import java.util.List;
 import javax.annotation.Nonnull;
+
+import org.apache.felix.scr.annotations.Activate;
 import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Property;
 import org.apache.felix.scr.annotations.Reference;
+import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.apache.felix.scr.annotations.ReferencePolicy;
+import org.apache.felix.scr.annotations.ReferencePolicyOption;
 import org.apache.felix.scr.annotations.Service;
+import org.apache.jackrabbit.oak.commons.PropertiesUtil;
+import org.apache.jackrabbit.oak.plugins.index.aggregate.AggregateIndexProvider;
+import org.apache.jackrabbit.oak.plugins.index.aggregate.NodeAggregator;
 import org.apache.jackrabbit.oak.plugins.index.solr.configuration.OakSolrConfigurationProvider;
 import org.apache.jackrabbit.oak.plugins.index.solr.query.SolrQueryIndexProvider;
 import org.apache.jackrabbit.oak.plugins.index.solr.server.SolrServerProvider;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
 import org.apache.jackrabbit.oak.spi.query.QueryIndexProvider;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
+import org.osgi.service.component.ComponentContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * Osgi Service that provides Solr based {@link org.apache.jackrabbit.oak.spi.query.QueryIndex}es
@@ -35,22 +47,47 @@ import org.apache.jackrabbit.oak.spi.sta
  * @see org.apache.jackrabbit.oak.plugins.index.solr.query.SolrQueryIndexProvider
  * @see QueryIndexProvider
  */
-@Component(metatype = false, immediate = true)
+@Component(metatype = true, immediate = true, label = "Oak Solr Query index provider configuration")
 @Service(value = QueryIndexProvider.class)
 public class SolrQueryIndexProviderService implements QueryIndexProvider {
 
+    private static final boolean QUERY_TIME_AGGREGATION_DEFAULT = false;
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
     @Reference
     private SolrServerProvider solrServerProvider;
 
     @Reference
     private OakSolrConfigurationProvider oakSolrConfigurationProvider;
 
+    @Reference(cardinality = ReferenceCardinality.OPTIONAL_UNARY,
+            policyOption = ReferencePolicyOption.GREEDY,
+            policy = ReferencePolicy.DYNAMIC
+    )
+    private NodeAggregator nodeAggregator;
+
+    @Property(boolValue = QUERY_TIME_AGGREGATION_DEFAULT, label = "query time aggregation",
+            description = "enable query time aggregation for Solr index")
+    private static final String QUERY_TIME_AGGREGATION = "query.aggregation";
+
+    private boolean queryTimeAggregation;
+
+    @Activate
+    protected void activate(ComponentContext componentContext) {
+        Object value = componentContext.getProperties().get(QUERY_TIME_AGGREGATION);
+        queryTimeAggregation = PropertiesUtil.toBoolean(value, QUERY_TIME_AGGREGATION_DEFAULT);
+    }
+
     @Override
     @Nonnull
     public List<? extends QueryIndex> getQueryIndexes(NodeState nodeState) {
         if (solrServerProvider != null && oakSolrConfigurationProvider != null) {
-            return new SolrQueryIndexProvider(solrServerProvider,
-                    oakSolrConfigurationProvider).getQueryIndexes(nodeState);
+            SolrQueryIndexProvider solrQueryIndexProvider = new SolrQueryIndexProvider(solrServerProvider,
+                    oakSolrConfigurationProvider, nodeAggregator);
+            log.debug("creating Solr query index provider {} query time aggregation", queryTimeAggregation ? "with" : "without");
+            return queryTimeAggregation ? AggregateIndexProvider.wrap(solrQueryIndexProvider).getQueryIndexes(nodeState) :
+                    solrQueryIndexProvider.getQueryIndexes(nodeState);
         } else {
             return new ArrayList<QueryIndex>();
         }

Modified: jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndex.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndex.java?rev=1646469&r1=1646468&r2=1646469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndex.java (original)
+++ jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndex.java Thu Dec 18 14:40:53 2014
@@ -16,7 +16,6 @@
  */
 package org.apache.jackrabbit.oak.plugins.index.solr.query;
 
-import java.io.IOException;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Deque;
@@ -29,6 +28,7 @@ import com.google.common.collect.Abstrac
 import com.google.common.collect.Queues;
 import com.google.common.collect.Sets;
 import org.apache.jackrabbit.oak.api.PropertyValue;
+import org.apache.jackrabbit.oak.commons.PathUtils;
 import org.apache.jackrabbit.oak.plugins.index.aggregate.NodeAggregator;
 import org.apache.jackrabbit.oak.plugins.index.solr.configuration.OakSolrConfiguration;
 import org.apache.jackrabbit.oak.query.QueryEngineSettings;
@@ -77,12 +77,15 @@ public class SolrQueryIndex implements F
 
     private final NodeAggregator aggregator;
 
-    public SolrQueryIndex(String name, SolrServer solrServer, OakSolrConfiguration configuration) {
+    public SolrQueryIndex(String name, SolrServer solrServer, OakSolrConfiguration configuration, NodeAggregator aggregator) {
         this.name = name;
         this.solrServer = solrServer;
         this.configuration = configuration;
-        // TODO this index should support aggregation in the same way as the Lucene index
-        this.aggregator = null;
+        this.aggregator = aggregator;
+    }
+
+    public SolrQueryIndex(String name, SolrServer solrServer, OakSolrConfiguration configuration) {
+        this(name, solrServer, configuration, null);
     }
 
     @Override
@@ -93,7 +96,9 @@ public class SolrQueryIndex implements F
     @Override
     public double getCost(Filter filter, NodeState root) {
         // cost is inverse proportional to the number of matching restrictions, infinite if no restriction matches
-        return 10d / getMatchingFilterRestrictions(filter);
+        double cost = 10d / getMatchingFilterRestrictions(filter);
+        log.debug("Solr: cost for {} is {}", name, cost);
+        return cost;
     }
 
     private int getMatchingFilterRestrictions(Filter filter) {
@@ -450,15 +455,15 @@ public class SolrQueryIndex implements F
     }
 
     @Override
-    public Cursor query(final Filter filter, NodeState root) {
+    public Cursor query(final Filter filter, final NodeState root) {
         Cursor cursor;
         try {
-            final Set<String> relPaths = filter.getFullTextConstraint() != null ? getRelativePaths(filter.getFullTextConstraint()) : Collections.<String>emptySet();
+            final Set<String> relPaths = filter.getFullTextConstraint() != null ? getRelativePaths(filter.getFullTextConstraint())
+                    : Collections.<String>emptySet();
             final String parent = relPaths.size() == 0 ? "" : relPaths.iterator().next();
 
             final int parentDepth = getDepth(parent);
 
-
             cursor = new SolrRowCursor(new AbstractIterator<SolrResultRow>() {
                 private final Set<String> seenPaths = Sets.newHashSet();
                 private final Deque<SolrResultRow> queue = Queues.newArrayDeque();
@@ -476,6 +481,7 @@ public class SolrQueryIndex implements F
                 private SolrResultRow convertToRow(SolrDocument doc) {
                     String path = String.valueOf(doc.getFieldValue(configuration.getPathField()));
                     if (path != null) {
+                        log.debug("converting path {}", path);
                         if ("".equals(path)) {
                             path = "/";
                         }
@@ -493,6 +499,7 @@ public class SolrQueryIndex implements F
                         if (scoreObj != null) {
                             score = (Float) scoreObj;
                         }
+                        log.debug("converted to {}", path);
                         return new SolrResultRow(path, score, doc);
                     }
                     return null;
@@ -550,14 +557,28 @@ public class SolrQueryIndex implements F
         return cursor;
     }
 
+    private boolean exists(SolrResultRow row, NodeState root) {
+        boolean result = true;
+        NodeState nodeState = root;
+        for (String n : PathUtils.elements(row.path)) {
+            if (nodeState.hasChildNode(n)) {
+                nodeState = nodeState.getChildNode(n);
+            }
+            else {
+                result = false;
+                break;
+            }
+        }
+        return result;
+    }
+
     static class SolrResultRow {
         final String path;
         final double score;
-        SolrDocument doc;
+        final SolrDocument doc;
 
         SolrResultRow(String path, double score) {
-            this.path = path;
-            this.score = score;
+            this(path, score, null);
         }
 
         SolrResultRow(String path, double score, SolrDocument doc) {

Modified: jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexProvider.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexProvider.java?rev=1646469&r1=1646468&r2=1646469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexProvider.java (original)
+++ jackrabbit/oak/trunk/oak-solr-core/src/main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexProvider.java Thu Dec 18 14:40:53 2014
@@ -21,6 +21,7 @@ import java.util.List;
 import javax.annotation.Nonnull;
 import org.apache.jackrabbit.oak.api.PropertyState;
 import org.apache.jackrabbit.oak.api.Type;
+import org.apache.jackrabbit.oak.plugins.index.aggregate.NodeAggregator;
 import org.apache.jackrabbit.oak.plugins.index.solr.configuration.OakSolrConfigurationProvider;
 import org.apache.jackrabbit.oak.plugins.index.solr.server.SolrServerProvider;
 import org.apache.jackrabbit.oak.spi.query.QueryIndex;
@@ -45,9 +46,17 @@ public class SolrQueryIndexProvider impl
 
     private final OakSolrConfigurationProvider oakSolrConfigurationProvider;
 
-    public SolrQueryIndexProvider(SolrServerProvider solrServerProvider, OakSolrConfigurationProvider oakSolrConfigurationProvider) {
+    private final NodeAggregator aggregator;
+
+    public SolrQueryIndexProvider(SolrServerProvider solrServerProvider, OakSolrConfigurationProvider oakSolrConfigurationProvider,
+                                  NodeAggregator nodeAggregator) {
         this.oakSolrConfigurationProvider = oakSolrConfigurationProvider;
         this.solrServerProvider = solrServerProvider;
+        this.aggregator = nodeAggregator;
+    }
+
+    public SolrQueryIndexProvider(SolrServerProvider solrServerProvider, OakSolrConfigurationProvider oakSolrConfigurationProvider) {
+        this(solrServerProvider, oakSolrConfigurationProvider, null);
     }
 
     @Nonnull
@@ -64,9 +73,6 @@ public class SolrQueryIndexProvider impl
             PropertyState type = definition.getProperty(TYPE_PROPERTY_NAME);
             if (type != null
                     && SolrQueryIndex.TYPE.equals(type.getValue(Type.STRING))) {
-                if (log.isDebugEnabled()) {
-                    log.debug("found a Solr index definition {}", entry.getName());
-                }
                 try {
                     SolrServer solrServer = solrServerProvider.getSolrServer();
                     // the query engine should be returned only if the server is alive, otherwise other indexes should be used
@@ -74,7 +80,8 @@ public class SolrQueryIndexProvider impl
                         tempIndexes.add(new SolrQueryIndex(
                                 entry.getName(),
                                 solrServer,
-                                oakSolrConfigurationProvider.getConfiguration()));
+                                oakSolrConfigurationProvider.getConfiguration(),
+                                aggregator));
                     }
                     else {
                         if (log.isWarnEnabled()) {

Modified: jackrabbit/oak/trunk/oak-solr-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexTest.java
URL: http://svn.apache.org/viewvc/jackrabbit/oak/trunk/oak-solr-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexTest.java?rev=1646469&r1=1646468&r2=1646469&view=diff
==============================================================================
--- jackrabbit/oak/trunk/oak-solr-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexTest.java (original)
+++ jackrabbit/oak/trunk/oak-solr-core/src/test/java/org/apache/jackrabbit/oak/plugins/index/solr/query/SolrQueryIndexTest.java Thu Dec 18 14:40:53 2014
@@ -29,6 +29,7 @@ import org.apache.jackrabbit.oak.query.Q
 import org.apache.jackrabbit.oak.query.ast.Operator;
 import org.apache.jackrabbit.oak.query.ast.SelectorImpl;
 import org.apache.jackrabbit.oak.query.index.FilterImpl;
+import org.apache.jackrabbit.oak.spi.query.Cursor;
 import org.apache.jackrabbit.oak.spi.query.Filter;
 import org.apache.jackrabbit.oak.spi.query.PropertyValues;
 import org.apache.jackrabbit.oak.spi.state.NodeState;
@@ -258,4 +259,29 @@ public class SolrQueryIndexTest {
         assertNotNull(plan);
         assertTrue(plan.contains("q=name%3Ahello")); // query gets converted to a fielded query on name field
     }
+
+    @Test
+    public void testUnion() throws Exception {
+        NodeState root = mock(NodeState.class);
+        when(root.getNames(any(String.class))).thenReturn(Collections.<String>emptySet());
+        SelectorImpl selector = new SelectorImpl(root, "a");
+        String sqlQuery = "select [jcr:path], [jcr:score], [rep:excerpt] from [nt:hierarchyNode] as a where" +
+                " isdescendantnode(a, '/content') and contains([jcr:content/*], 'founded') union select [jcr:path]," +
+                " [jcr:score], [rep:excerpt] from [nt:hierarchyNode] as a where isdescendantnode(a, '/content') and " +
+                "contains([jcr:content/jcr:title], 'founded') union select [jcr:path], [jcr:score], [rep:excerpt]" +
+                " from [nt:hierarchyNode] as a where isdescendantnode(a, '/content') and " +
+                "contains([jcr:content/jcr:description], 'founded') order by [jcr:score] desc";
+        SolrServer solrServer = TestUtils.createSolrServer();
+        OakSolrConfiguration configuration = new DefaultSolrConfiguration() {
+            @Override
+            public boolean useForPropertyRestrictions() {
+                return true;
+            }
+        };
+        SolrQueryIndex solrQueryIndex = new SolrQueryIndex("solr", solrServer, configuration);
+        FilterImpl filter = new FilterImpl(selector, sqlQuery, new QueryEngineSettings());
+        Cursor cursor = solrQueryIndex.query(filter, root);
+        assertNotNull(cursor);
+
+    }
 }



Re: svn commit: r1646469 - in /jackrabbit/oak/trunk/oak-solr-core/src: main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/ main/java/org/apache/jackrabbit/oak/plugins/index/solr/osgi/ main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/ ...

Posted by Tommaso Teofili <to...@gmail.com>.
Hi Chetan,

I agree index time aggregation will be probably faster but I'd like the
Solr index to be able to do both so that it's on par with the Lucene index
features. On a side note the mentioned commit would only work pre-OAK-2119
so the next step will be (I'm currently working on it) to properly move to
implement the AdvanceQueryIndex.

On a related note it'd be probably good to add some benchmarks on query
time / index time aggregation performances for Lucene (and for Solr once
available) to have a proper evidence of what we expect.

Regards,
Tommaso

2014-12-19 6:33 GMT+01:00 Chetan Mehrotra <ch...@gmail.com>:
>
> Hi Tommaso,
>
> On Thu, Dec 18, 2014 at 8:10 PM,  <to...@apache.org> wrote:
> > Log:
> > OAK-2168 - support qt aggregation (wip)
>
> I think query time aggregation over a remote index like Solr would not
> be able to perform well. Specially for multi term queries. So it might
> be better if we only add support for index time aggregation.
>
> Chetan Mehrotra
>

Re: svn commit: r1646469 - in /jackrabbit/oak/trunk/oak-solr-core/src: main/java/org/apache/jackrabbit/oak/plugins/index/solr/index/ main/java/org/apache/jackrabbit/oak/plugins/index/solr/osgi/ main/java/org/apache/jackrabbit/oak/plugins/index/solr/query/ ...

Posted by Chetan Mehrotra <ch...@gmail.com>.
Hi Tommaso,

On Thu, Dec 18, 2014 at 8:10 PM,  <to...@apache.org> wrote:
> Log:
> OAK-2168 - support qt aggregation (wip)

I think query time aggregation over a remote index like Solr would not
be able to perform well. Specially for multi term queries. So it might
be better if we only add support for index time aggregation.

Chetan Mehrotra