You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2015/01/01 15:46:55 UTC

svn commit: r1648848 - in /lucene/dev/branches/branch_5x/solr: ./ core/ core/src/java/org/apache/solr/handler/BlobHandler.java core/src/java/org/apache/solr/handler/SolrConfigHandler.java

Author: noble
Date: Thu Jan  1 14:46:55 2015
New Revision: 1648848

URL: http://svn.apache.org/r1648848
Log:
SOLR-6787 changed the schema of blob store. now, the id is blobName/version and not the md5

Modified:
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/BlobHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/BlobHandler.java?rev=1648848&r1=1648847&r2=1648848&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/BlobHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/BlobHandler.java Thu Jan  1 14:46:55 2015
@@ -37,17 +37,14 @@ import org.apache.lucene.search.TopDocs;
 import org.apache.lucene.search.TopFieldDocs;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.params.UpdateParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
@@ -86,6 +83,11 @@ public class BlobHandler extends Request
         rsp.add("error","Name not found");
         return;
       }
+      String err = SolrConfigHandler.validateName(blobName);
+      if(err!=null){
+        rsp.add("error", err);
+        return;
+      }
       if(req.getContentStreams() == null )  {
         rsp.add("error","No stream");
         return;
@@ -98,12 +100,12 @@ public class BlobHandler extends Request
         m.update(payload.array(),payload.position(),payload.limit());
         String md5 = new BigInteger(1,m.digest()).toString(16);
 
-        TopDocs duplicate = req.getSearcher().search(new TermQuery(new Term("id", md5)), 1);
+        TopDocs duplicate = req.getSearcher().search(new TermQuery(new Term("md5", md5)), 1);
         if(duplicate.totalHits >0){
           rsp.add("error", "duplicate entry");
           req.forward(null,
               new MapSolrParams((Map) makeMap(
-              "q", "id:" + md5,
+              "q", "md5:" + md5,
               "fl", "id,size,version,timestamp,blobName")),
               rsp);
           return;
@@ -118,11 +120,13 @@ public class BlobHandler extends Request
           Number n = doc.getField("version").numericValue();
           version = n.longValue();
         }
-
+        version++;
+        String id = blobName+"/"+version;
         indexMap(req, makeMap(
-            "id", md5,
+            "id", id,
+            "md5", md5,
             "blobName", blobName,
-            "version", ++version,
+            "version", version,
             "timestamp", new Date(),
             "size", payload.limit(),
             "blob", payload));
@@ -145,7 +149,7 @@ public class BlobHandler extends Request
           throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "Please send the request in the format /blob/<blobName>/<version>");
         } else {
           String q = "blobName:{0}";
-          if(version!=-1) q+= " AND version:{1}";
+          if(version != -1) q = "id:{0}/{1}";
           QParser qparser =  QParser.getParser(MessageFormat.format(q,blobName,version) , "lucene", req);
           final TopDocs docs = req.getSearcher().search(qparser.parse(), 1, new Sort( new SortField("version", SortField.Type.LONG, true)));
           if(docs.totalHits>0){
@@ -174,12 +178,16 @@ public class BlobHandler extends Request
         }
       } else {
         String q = "*:*";
-        if (blobName != null) q = "blobName" + ":" + blobName;
-        if (version > -1) q += " AND version:" + version;
+        if(blobName != null){
+          q = "blobName:{0}";
+          if(version != -1){
+            q = "id:{0}/{1}";
+          }
+        }
 
         req.forward(null,
             new MapSolrParams((Map) makeMap(
-                "q", q,
+                "q", MessageFormat.format(q,blobName,version),
                 "fl", "id,size,version,timestamp,blobName",
                 "sort", "version desc"))
             ,rsp);
@@ -220,8 +228,9 @@ public class BlobHandler extends Request
       "  <fieldType name='bytes' class='solr.BinaryField'/>\n" +
       "  <fieldType name='date' class='solr.TrieDateField'/>\n" +
       "  <field name='id'   type='string'   indexed='true'  stored='true'  multiValued='false' required='true'/>\n" +
+      "  <field name='md5'   type='string'   indexed='true'  stored='true'  multiValued='false' required='true'/>\n" +
       "  <field name='blob'      type='bytes'   indexed='false' stored='true'  multiValued='false' />\n" +
-      "  <field name='size'      type='long'   indexed='false' stored='true'  multiValued='false' />\n" +
+      "  <field name='size'      type='long'   indexed='true' stored='true'  multiValued='false' />\n" +
       "  <field name='version'   type='long'     indexed='true'  stored='true'  multiValued='false' />\n" +
       "  <field name='timestamp'   type='date'   indexed='true'  stored='true'  multiValued='false' />\n" +
       "  <field name='blobName'      type='string'   indexed='true'  stored='true'  multiValued='false' />\n" +

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java?rev=1648848&r1=1648847&r2=1648848&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/handler/SolrConfigHandler.java Thu Jan  1 14:46:55 2015
@@ -32,7 +32,6 @@ import java.util.Set;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.MapSolrParams;
@@ -40,21 +39,15 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.core.ConfigOverlay;
-import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.PluginInfo;
 import org.apache.solr.core.RequestParams;
 import org.apache.solr.core.SolrConfig;
-import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
-import org.apache.solr.schema.ManagedIndexSchema;
 import org.apache.solr.schema.SchemaManager;
 import org.apache.solr.util.CommandOperation;
-import org.apache.solr.util.plugin.SolrCoreAware;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -191,8 +184,9 @@ public class SolrConfigHandler extends R
                 continue;
               }
               key = key.trim();
-              if (!validName(key)) {
-                op.addError(MessageFormat.format("''{0}'' name should only have chars [a-zA-Z_-.0-9] ", key));
+              String err = validateName(key);
+              if (err !=null) {
+                op.addError(err);
                 continue;
               }
 
@@ -395,7 +389,7 @@ public class SolrConfigHandler extends R
 
   }
 
-  public static boolean validName(String s) {
+  public static String validateName(String s) {
     for(int i=0;i<s.length();i++) {
       char c = s.charAt(i);
       if((c >= 'A' && c<='Z') ||
@@ -405,9 +399,11 @@ public class SolrConfigHandler extends R
            c == '-'||
            c == '.'
           ) continue;
-      else return false;
+      else {
+        return MessageFormat.format("''{0}'' name should only have chars [a-zA-Z_-.0-9] ",s);
+      }
     }
-    return true;
+    return null;
   }
 
   static void setWt(SolrQueryRequest req, String wt){