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 2016/02/23 11:45:34 UTC

[1/2] lucene-solr git commit: SOLR-8719 renamed classes to make their usage generic

Repository: lucene-solr
Updated Branches:
  refs/heads/master a77d67a92 -> a33236aa0


SOLR-8719 renamed classes to make their usage generic


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/4d9d0c00
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/4d9d0c00
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/4d9d0c00

Branch: refs/heads/master
Commit: 4d9d0c0011de8653fbe404af702e822883e8fb13
Parents: cf96432
Author: Noble Paul <no...@apache.org>
Authored: Tue Feb 23 16:14:49 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Tue Feb 23 16:14:49 2016 +0530

----------------------------------------------------------------------
 .../org/apache/solr/core/BlobRepository.java    | 254 +++++++++++++++++++
 .../org/apache/solr/core/CoreContainer.java     |   6 +-
 .../org/apache/solr/core/JarRepository.java     | 222 ----------------
 .../java/org/apache/solr/core/PluginBag.java    |  10 +-
 4 files changed, 262 insertions(+), 230 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d9d0c00/solr/core/src/java/org/apache/solr/core/BlobRepository.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/BlobRepository.java b/solr/core/src/java/org/apache/solr/core/BlobRepository.java
new file mode 100644
index 0000000..6739826
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/core/BlobRepository.java
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * 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.
+ */
+package org.apache.solr.core;
+
+import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
+import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.lang.invoke.MethodHandles;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipInputStream;
+
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.util.CryptoKeys;
+import org.apache.solr.util.SimplePostTool;
+import org.apache.zookeeper.server.ByteBufferInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The purpose of this class is to store the Jars loaded in memory and to keep only one copy of the Jar in a single node.
+ */
+public class BlobRepository {
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  static final Random RANDOM;
+
+  static {
+    // We try to make things reproducible in the context of our tests by initializing the random instance
+    // based on the current seed
+    String seed = System.getProperty("tests.seed");
+    if (seed == null) {
+      RANDOM = new Random();
+    } else {
+      RANDOM = new Random(seed.hashCode());
+    }
+  }
+
+  private final CoreContainer coreContainer;
+  private Map<String, BlobContent> blobs = new ConcurrentHashMap<>();
+
+  public BlobRepository(CoreContainer coreContainer) {
+    this.coreContainer = coreContainer;
+  }
+
+  public static ByteBuffer getFileContent(BlobContent blobContent, String entryName) throws IOException {
+    ByteArrayInputStream zipContents = new ByteArrayInputStream(blobContent.buffer.array(), blobContent.buffer.arrayOffset(), blobContent.buffer.limit());
+    ZipInputStream zis = new ZipInputStream(zipContents);
+    try {
+      ZipEntry entry;
+      while ((entry = zis.getNextEntry()) != null) {
+        if (entryName == null || entryName.equals(entry.getName())) {
+          SimplePostTool.BAOS out = new SimplePostTool.BAOS();
+          byte[] buffer = new byte[2048];
+          int size;
+          while ((size = zis.read(buffer, 0, buffer.length)) != -1) {
+            out.write(buffer, 0, size);
+          }
+          out.close();
+          return out.getByteBuffer();
+        }
+      }
+    } finally {
+      zis.closeEntry();
+    }
+    return null;
+  }
+
+  /**
+   * Returns the contents of a jar and increments a reference count. Please return the same object to decrease the refcount
+   *
+   * @param key it is a combination of blobname and version like blobName/version
+   * @return The reference of a jar
+   */
+  public BlobContentRef getBlobIncRef(String key) {
+    BlobContent aBlob = blobs.get(key);
+    if (aBlob == null) {
+      if (this.coreContainer.isZooKeeperAware()) {
+        Replica replica = getSystemCollReplica();
+        String url = replica.getStr(BASE_URL_PROP) + "/.system/blob/" + key + "?wt=filestream";
+
+        HttpClient httpClient = coreContainer.getUpdateShardHandler().getHttpClient();
+        HttpGet httpGet = new HttpGet(url);
+        ByteBuffer b;
+        try {
+          HttpResponse entity = httpClient.execute(httpGet);
+          int statusCode = entity.getStatusLine().getStatusCode();
+          if (statusCode != 200) {
+            throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no such blob or version available: " + key);
+          }
+          b = SimplePostTool.inputStreamToByteArray(entity.getEntity().getContent());
+        } catch (Exception e) {
+          if (e instanceof SolrException) {
+            throw (SolrException) e;
+          } else {
+            throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "could not load : " + key, e);
+          }
+        } finally {
+          httpGet.releaseConnection();
+        }
+        blobs.put(key, aBlob = new BlobContent(key, b));
+      } else {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Jar loading is not supported in non-cloud mode");
+        // todo
+      }
+
+    }
+
+    BlobContentRef ref = new BlobContentRef(aBlob);
+    synchronized (aBlob.references) {
+      aBlob.references.add(ref);
+    }
+    return ref;
+
+  }
+
+  private Replica getSystemCollReplica() {
+    ZkStateReader zkStateReader = this.coreContainer.getZkController().getZkStateReader();
+    ClusterState cs = zkStateReader.getClusterState();
+    DocCollection coll = cs.getCollectionOrNull(CollectionsHandler.SYSTEM_COLL);
+    if (coll == null) throw new SolrException(SERVICE_UNAVAILABLE, ".system collection not available");
+    ArrayList<Slice> slices = new ArrayList<>(coll.getActiveSlices());
+    if (slices.isEmpty()) throw new SolrException(SERVICE_UNAVAILABLE, "No active slices for .system collection");
+    Collections.shuffle(slices, RANDOM); //do load balancing
+
+    Replica replica = null;
+    for (Slice slice : slices) {
+      List<Replica> replicas = new ArrayList<>(slice.getReplicasMap().values());
+      Collections.shuffle(replicas, RANDOM);
+      for (Replica r : replicas) {
+        if (r.getState() == Replica.State.ACTIVE) {
+          if(zkStateReader.getClusterState().getLiveNodes().contains(r.get(ZkStateReader.NODE_NAME_PROP))){
+            replica = r;
+            break;
+          } else {
+            log.info("replica {} says it is active but not a member of live nodes", r.get(ZkStateReader.NODE_NAME_PROP));
+          }
+        }
+      }
+    }
+    if (replica == null) {
+      throw new SolrException(SERVICE_UNAVAILABLE, ".no active replica available for .system collection");
+    }
+    return replica;
+  }
+
+  /**
+   * This is to decrement a ref count
+   *
+   * @param ref The reference that is already there. Doing multiple calls with same ref will not matter
+   */
+  public void decrementBlobRefCount(BlobContentRef ref) {
+    if (ref == null) return;
+    synchronized (ref.blob.references) {
+      if (!ref.blob.references.remove(ref)) {
+        log.error("Multiple releases for the same reference");
+      }
+      if (ref.blob.references.isEmpty()) {
+        blobs.remove(ref.blob.key);
+      }
+    }
+
+  }
+
+  public static class BlobContent {
+    private final String key;
+    private Map<String, Object> decodedObjects = null;
+    // TODO move this off-heap
+    private final ByteBuffer buffer;
+    // ref counting mechanism
+    private final Set<BlobContentRef> references = new HashSet<>();
+
+
+    public BlobContent(String key, ByteBuffer buffer) {
+      this.key = key;
+      this.buffer = buffer;
+    }
+
+    /**
+     * This method decodes the byte[] to a custom Object
+     *
+     * @param key     The key is used to store the decoded Object. it is possible to have multiple
+     *                decoders for the same blob (may be unusual).
+     * @param decoder A decoder instance
+     * @return the decoded Object . If it was already decoded, then return from the cache
+     */
+    public <T> T decodeAndCache(String key, Decoder<T> decoder) {
+      if (decodedObjects == null) {
+        synchronized (this) {
+          if (decodedObjects == null) decodedObjects = new ConcurrentHashMap<>();
+        }
+      }
+
+      Object t = decodedObjects.get(key);
+      if (t != null) return (T) t;
+      t = decoder.decode(new ByteBufferInputStream(buffer));
+      decodedObjects.put(key, t);
+      return (T) t;
+
+    }
+
+    public String checkSignature(String base64Sig, CryptoKeys keys) {
+      return keys.verify(base64Sig, buffer);
+    }
+
+  }
+
+  public interface Decoder<T> {
+
+    T decode(InputStream inputStream);
+  }
+
+  public static class BlobContentRef {
+    public final BlobContent blob;
+
+    private BlobContentRef(BlobContent blob) {
+      this.blob = blob;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d9d0c00/solr/core/src/java/org/apache/solr/core/CoreContainer.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 7703721..7a55e05 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -134,7 +134,7 @@ public class CoreContainer {
 
   private String hostName;
 
-  private final JarRepository jarRepository = new JarRepository(this);
+  private final BlobRepository blobRepository = new BlobRepository(this);
 
   private PluginBag<SolrRequestHandler> containerHandlers = new PluginBag<>(SolrRequestHandler.class, null);
 
@@ -1098,8 +1098,8 @@ public class CoreContainer {
     return core;
   }
 
-  public JarRepository getJarRepository(){
-    return jarRepository;
+  public BlobRepository getBlobRepository(){
+    return blobRepository;
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d9d0c00/solr/core/src/java/org/apache/solr/core/JarRepository.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/JarRepository.java b/solr/core/src/java/org/apache/solr/core/JarRepository.java
deleted file mode 100644
index c760b59..0000000
--- a/solr/core/src/java/org/apache/solr/core/JarRepository.java
+++ /dev/null
@@ -1,222 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * 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.
- */
-package org.apache.solr.core;
-
-import static org.apache.solr.common.SolrException.ErrorCode.SERVICE_UNAVAILABLE;
-import static org.apache.solr.common.cloud.ZkStateReader.BASE_URL_PROP;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.lang.invoke.MethodHandles;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-
-import org.apache.http.HttpResponse;
-import org.apache.http.client.HttpClient;
-import org.apache.http.client.methods.HttpGet;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.cloud.ClusterState;
-import org.apache.solr.common.cloud.DocCollection;
-import org.apache.solr.common.cloud.Replica;
-import org.apache.solr.common.cloud.Slice;
-import org.apache.solr.common.cloud.ZkStateReader;
-import org.apache.solr.handler.admin.CollectionsHandler;
-import org.apache.solr.util.CryptoKeys;
-import org.apache.solr.util.SimplePostTool;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The purpose of this class is to store the Jars loaded in memory and to keep only one copy of the Jar in a single node.
- */
-public class JarRepository {
-  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
-  static final Random RANDOM;
-
-  static {
-    // We try to make things reproducible in the context of our tests by initializing the random instance
-    // based on the current seed
-    String seed = System.getProperty("tests.seed");
-    if (seed == null) {
-      RANDOM = new Random();
-    } else {
-      RANDOM = new Random(seed.hashCode());
-    }
-  }
-
-  private final CoreContainer coreContainer;
-  private Map<String, JarContent> jars = new ConcurrentHashMap<>();
-
-  public JarRepository(CoreContainer coreContainer) {
-    this.coreContainer = coreContainer;
-  }
-
-  /**
-   * Returns the contents of a jar and increments a reference count. Please return the same object to decrease the refcount
-   *
-   * @param key it is a combination of blobname and version like blobName/version
-   * @return The reference of a jar
-   */
-  public JarContentRef getJarIncRef(String key) {
-    JarContent jar = jars.get(key);
-    if (jar == null) {
-      if (this.coreContainer.isZooKeeperAware()) {
-        Replica replica = getSystemCollReplica();
-        String url = replica.getStr(BASE_URL_PROP) + "/.system/blob/" + key + "?wt=filestream";
-
-        HttpClient httpClient = coreContainer.getUpdateShardHandler().getHttpClient();
-        HttpGet httpGet = new HttpGet(url);
-        ByteBuffer b;
-        try {
-          HttpResponse entity = httpClient.execute(httpGet);
-          int statusCode = entity.getStatusLine().getStatusCode();
-          if (statusCode != 200) {
-            throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "no such blob or version available: " + key);
-          }
-          b = SimplePostTool.inputStreamToByteArray(entity.getEntity().getContent());
-        } catch (Exception e) {
-          if (e instanceof SolrException) {
-            throw (SolrException) e;
-          } else {
-            throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "could not load : " + key, e);
-          }
-        } finally {
-          httpGet.releaseConnection();
-        }
-        jars.put(key, jar = new JarContent(key, b));
-      } else {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Jar loading is not supported in non-cloud mode");
-        // todo
-      }
-
-    }
-
-    JarContentRef ref = new JarContentRef(jar);
-    synchronized (jar.references) {
-      jar.references.add(ref);
-    }
-    return ref;
-
-  }
-
-  private Replica getSystemCollReplica() {
-    ZkStateReader zkStateReader = this.coreContainer.getZkController().getZkStateReader();
-    ClusterState cs = zkStateReader.getClusterState();
-    DocCollection coll = cs.getCollectionOrNull(CollectionsHandler.SYSTEM_COLL);
-    if (coll == null) throw new SolrException(SERVICE_UNAVAILABLE, ".system collection not available");
-    ArrayList<Slice> slices = new ArrayList<>(coll.getActiveSlices());
-    if (slices.isEmpty()) throw new SolrException(SERVICE_UNAVAILABLE, "No active slices for .system collection");
-    Collections.shuffle(slices, RANDOM); //do load balancing
-
-    Replica replica = null;
-    for (Slice slice : slices) {
-      List<Replica> replicas = new ArrayList<>(slice.getReplicasMap().values());
-      Collections.shuffle(replicas, RANDOM);
-      for (Replica r : replicas) {
-        if (r.getState() == Replica.State.ACTIVE) {
-          if(zkStateReader.getClusterState().getLiveNodes().contains(r.get(ZkStateReader.NODE_NAME_PROP))){
-            replica = r;
-            break;
-          } else {
-            log.info("replica {} says it is active but not a member of live nodes", r.get(ZkStateReader.NODE_NAME_PROP));
-          }
-        }
-      }
-    }
-    if (replica == null) {
-      throw new SolrException(SERVICE_UNAVAILABLE, ".no active replica available for .system collection");
-    }
-    return replica;
-  }
-
-  /**
-   * This is to decrement a ref count
-   *
-   * @param ref The reference that is already there. Doing multiple calls with same ref will not matter
-   */
-  public void decrementJarRefCount(JarContentRef ref) {
-    if (ref == null) return;
-    synchronized (ref.jar.references) {
-      if (!ref.jar.references.remove(ref)) {
-        log.error("Multiple releases for the same reference");
-      }
-      if (ref.jar.references.isEmpty()) {
-        jars.remove(ref.jar.key);
-      }
-    }
-
-  }
-
-  public static class JarContent {
-    private final String key;
-    // TODO move this off-heap
-    private final ByteBuffer buffer;
-    // ref counting mechanism
-    private final Set<JarContentRef> references = new HashSet<>();
-
-    public JarContent(String key, ByteBuffer buffer) {
-      this.key = key;
-      this.buffer = buffer;
-    }
-
-    public ByteBuffer getFileContent(String entryName) throws IOException {
-      ByteArrayInputStream zipContents = new ByteArrayInputStream(buffer.array(), buffer.arrayOffset(), buffer.limit());
-      ZipInputStream zis = new ZipInputStream(zipContents);
-      try {
-        ZipEntry entry;
-        while ((entry = zis.getNextEntry()) != null) {
-          if (entryName == null || entryName.equals(entry.getName())) {
-            SimplePostTool.BAOS out = new SimplePostTool.BAOS();
-            byte[] buffer = new byte[2048];
-            int size;
-            while ((size = zis.read(buffer, 0, buffer.length)) != -1) {
-              out.write(buffer, 0, size);
-            }
-            out.close();
-            return out.getByteBuffer();
-          }
-        }
-      } finally {
-        zis.closeEntry();
-      }
-      return null;
-    }
-
-    public String checkSignature(String base64Sig, CryptoKeys keys) {
-      return keys.verify(base64Sig, buffer);
-    }
-
-  }
-
-  public static class JarContentRef {
-    public final JarContent jar;
-
-    private JarContentRef(JarContent jar) {
-      this.jar = jar;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4d9d0c00/solr/core/src/java/org/apache/solr/core/PluginBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/PluginBag.java b/solr/core/src/java/org/apache/solr/core/PluginBag.java
index 454633d..0defdad 100644
--- a/solr/core/src/java/org/apache/solr/core/PluginBag.java
+++ b/solr/core/src/java/org/apache/solr/core/PluginBag.java
@@ -386,7 +386,7 @@ public class PluginBag<T> implements AutoCloseable {
    */
   public static class RuntimeLib implements PluginInfoInitialized, AutoCloseable {
     private String name, version, sig;
-    private JarRepository.JarContentRef jarContent;
+    private BlobRepository.BlobContentRef jarContent;
     private final CoreContainer coreContainer;
     private boolean verified = false;
 
@@ -410,7 +410,7 @@ public class PluginBag<T> implements AutoCloseable {
       if (jarContent != null) return;
       synchronized (this) {
         if (jarContent != null) return;
-        jarContent = coreContainer.getJarRepository().getJarIncRef(name + "/" + version);
+        jarContent = coreContainer.getBlobRepository().getBlobIncRef(name + "/" + version);
       }
     }
 
@@ -430,13 +430,13 @@ public class PluginBag<T> implements AutoCloseable {
     public ByteBuffer getFileContent(String entryName) throws IOException {
       if (jarContent == null)
         throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "jar not available: " + name + "/" + version);
-      return jarContent.jar.getFileContent(entryName);
+      return BlobRepository.getFileContent(jarContent.blob, entryName);
 
     }
 
     @Override
     public void close() throws Exception {
-      if (jarContent != null) coreContainer.getJarRepository().decrementJarRefCount(jarContent);
+      if (jarContent != null) coreContainer.getBlobRepository().decrementBlobRefCount(jarContent);
     }
 
     public static List<RuntimeLib> getLibObjects(SolrCore core, List<PluginInfo> libs) {
@@ -472,7 +472,7 @@ public class PluginBag<T> implements AutoCloseable {
       }
 
       try {
-        String matchedKey = jarContent.jar.checkSignature(sig, new CryptoKeys(keys));
+        String matchedKey = jarContent.blob.checkSignature(sig, new CryptoKeys(keys));
         if (matchedKey == null)
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "No key matched signature for jar : " + name + " version: " + version);
         log.info("Jar {} signed with {} successfully verified", name, matchedKey);


[2/2] lucene-solr git commit: Merge remote-tracking branch 'origin/master'

Posted by no...@apache.org.
Merge remote-tracking branch 'origin/master'


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/a33236aa
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/a33236aa
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/a33236aa

Branch: refs/heads/master
Commit: a33236aa02b3cc71d45974f2f0fefafbe1cf7a6b
Parents: 4d9d0c0 a77d67a
Author: Noble Paul <no...@apache.org>
Authored: Tue Feb 23 16:15:28 2016 +0530
Committer: Noble Paul <no...@apache.org>
Committed: Tue Feb 23 16:15:28 2016 +0530

----------------------------------------------------------------------
 lucene/ivy-versions.properties                  |   4 +-
 solr/contrib/clustering/ivy.xml                 |   3 -
 solr/licenses/attributes-binder-1.2.3.jar.sha1  |   1 -
 solr/licenses/attributes-binder-1.3.1.jar.sha1  |   1 +
 solr/licenses/mahout-collections-1.0.jar.sha1   |   1 -
 .../licenses/mahout-collections-LICENSE-ASL.txt | 202 -------------------
 solr/licenses/mahout-collections-NOTICE.txt     |  56 -----
 solr/licenses/mahout-math-0.6.jar.sha1          |   1 -
 solr/licenses/mahout-math-LICENSE-ASL.txt       | 202 -------------------
 solr/licenses/mahout-math-NOTICE.txt            |  56 -----
 10 files changed, 2 insertions(+), 525 deletions(-)
----------------------------------------------------------------------