You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2013/01/18 19:31:23 UTC

svn commit: r1435287 [36/41] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/analysis/icu/ dev-tools/maven/ dev-tools/maven/lucene/benchmark/ dev-tools/maven/solr/ dev-tools/...

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/TestHashPartitioner.java Fri Jan 18 18:30:54 2013
@@ -17,16 +17,26 @@ package org.apache.solr.cloud;
  * the License.
  */
 
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.cloud.HashPartitioner;
-import org.apache.solr.common.cloud.HashPartitioner.Range;
+import org.apache.solr.common.cloud.CompositeIdRouter;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.DocRouter;
+import org.apache.solr.common.cloud.DocRouter.Range;
+import org.apache.solr.common.cloud.PlainIdRouter;
+import org.apache.solr.common.cloud.Slice;
+import org.apache.solr.common.util.Hash;
+import org.apache.solr.common.util.StrUtils;
 
 public class TestHashPartitioner extends SolrTestCaseJ4 {
   
   public void testMapHashes() throws Exception {
-    HashPartitioner hp = new HashPartitioner();
+    DocRouter hp = DocRouter.DEFAULT;
     List<Range> ranges;
 
     // make sure the partitioner uses the "natural" boundaries and doesn't suffer from an off-by-one
@@ -37,13 +47,22 @@ public class TestHashPartitioner extends
     assertEquals(0x00000000, ranges.get(1).min);
     assertEquals(0x7fffffff, ranges.get(1).max);
 
-    ranges = hp.partitionRange(2, 0, 0x7fffffff);
+    ranges = hp.partitionRange(2, new DocRouter.Range(0, 0x7fffffff));
     assertEquals(0x00000000, ranges.get(0).min);
     assertEquals(0x3fffffff, ranges.get(0).max);
     assertEquals(0x40000000, ranges.get(1).min);
     assertEquals(0x7fffffff, ranges.get(1).max);
 
-    for (int i = 1; i <= 30000; i += 13) {
+    int defaultLowerBits = 0x0000ffff;
+
+    for (int i = 1; i <= 30000; i++) {
+      // start skipping at higher numbers
+      if (i > 100) i+=13;
+      else if (i > 1000) i+=31;
+      else if (i > 5000) i+=101;
+
+      long rangeSize = 0x0000000100000000L / i;
+
       ranges = hp.partitionRange(i, hp.fullRange());
       assertEquals(i, ranges.size());
       assertTrue("First range does not start before " + Integer.MIN_VALUE
@@ -59,8 +78,196 @@ public class TestHashPartitioner extends
         assertEquals(range, newRange);
       }
 
+      // ensure that ranges are contiguous and that size deviations are not too large.
+      int lastEnd = Integer.MIN_VALUE - 1;
+      for (Range range : ranges) {
+        int currStart = range.min;
+        int currEnd = range.max;
+        assertEquals(lastEnd+1, currStart);
+
+        if (ranges.size() < 4000) {
+          // ranges should be rounded to avoid crossing hash domains
+          assertEquals(defaultLowerBits, currEnd & defaultLowerBits);
+
+          // given our rounding condition that domains should be less than 1/16 of the step size,
+          // this means that any sizing deviations should also be less than 1/16th of the idealized range size.
+          // boolean round = rangeStep >= (1<<bits)*16;
+
+          long currRangeSize = (long)currEnd - (long)currStart;
+          long error = Math.abs(rangeSize - currRangeSize);
+          assertTrue( error < rangeSize/16);
+        }
+
+
+        // String s = range.toString();
+        // Range newRange = hp.fromString(s);
+        // assertEquals(range, newRange);
+        lastEnd = currEnd;
+      }
 
     }
   }
-  
+
+  public int hash(String id) {
+    // our hashing is defined to be murmurhash3 on the UTF-8 bytes of the key.
+    return Hash.murmurhash3_x86_32(id, 0, id.length(), 0);
+  }
+
+  public void testHashCodes() throws Exception {
+    DocRouter router = DocRouter.getDocRouter(PlainIdRouter.NAME);
+    assertTrue(router instanceof PlainIdRouter);
+    DocCollection coll = createCollection(4, router);
+    doNormalIdHashing(coll);
+  }
+
+  public void doNormalIdHashing(DocCollection coll) throws Exception {
+    assertEquals(4, coll.getSlices().size());
+
+    doId(coll, "b", "shard1");
+    doId(coll, "c", "shard2");
+    doId(coll, "d", "shard3");
+    doId(coll, "e", "shard4");
+  }
+
+  public void doId(DocCollection coll, String id, String expectedShard) {
+    doIndex(coll, id, expectedShard);
+    doQuery(coll, id, expectedShard);
+  }
+
+  public void doIndex(DocCollection coll, String id, String expectedShard) {
+    DocRouter router = coll.getRouter();
+    Slice target = router.getTargetSlice(id, null, null, coll);
+    assertEquals(expectedShard, target.getName());
+  }
+
+  public void doQuery(DocCollection coll, String id, String expectedShards) {
+    DocRouter router = coll.getRouter();
+    Collection<Slice> slices = router.getSearchSlices(id, null, coll);
+
+    List<String> expectedShardStr = StrUtils.splitSmart(expectedShards, ",", true);
+
+    HashSet<String> expectedSet = new HashSet<String>(expectedShardStr);
+    HashSet<String> obtainedSet = new HashSet<String>();
+    for (Slice slice : slices) {
+      obtainedSet.add(slice.getName());
+    }
+
+    assertEquals(slices.size(), obtainedSet.size());  // make sure no repeated slices
+    assertEquals(expectedSet, obtainedSet);
+  }
+
+  public void testCompositeHashCodes() throws Exception {
+    DocRouter router = DocRouter.getDocRouter(CompositeIdRouter.NAME);
+    assertTrue(router instanceof CompositeIdRouter);
+    router = DocRouter.DEFAULT;
+    assertTrue(router instanceof CompositeIdRouter);
+
+    DocCollection coll = createCollection(4, router);
+    doNormalIdHashing(coll);
+
+    // ensure that the shard hashed to is only dependent on the first part of the compound key
+    doId(coll, "b!foo", "shard1");
+    doId(coll, "c!bar", "shard2");
+    doId(coll, "d!baz", "shard3");
+    doId(coll, "e!qux", "shard4");
+
+    // syntax to specify bits.
+    // Anything over 2 bits should give the same results as above (since only top 2 bits
+    // affect our 4 slice collection).
+    doId(coll, "b/2!foo", "shard1");
+    doId(coll, "c/2!bar", "shard2");
+    doId(coll, "d/2!baz", "shard3");
+    doId(coll, "e/2!qux", "shard4");
+
+    doId(coll, "b/32!foo", "shard1");
+    doId(coll, "c/32!bar", "shard2");
+    doId(coll, "d/32!baz", "shard3");
+    doId(coll, "e/32!qux", "shard4");
+
+    // no bits allocated to the first part (kind of odd why anyone would do that though)
+    doIndex(coll, "foo/0!b", "shard1");
+    doIndex(coll, "foo/0!c", "shard2");
+    doIndex(coll, "foo/0!d", "shard3");
+    doIndex(coll, "foo/0!e", "shard4");
+
+    // means cover whole range on the query side
+    doQuery(coll, "foo/0!", "shard1,shard2,shard3,shard4");
+
+    doQuery(coll, "b/1!", "shard1,shard2");   // top bit of hash(b)==1, so shard1 and shard2
+    doQuery(coll, "d/1!", "shard3,shard4");   // top bit of hash(b)==0, so shard3 and shard4
+  }
+
+  /***
+  public void testPrintHashCodes() throws Exception {
+   // from negative to positive, the upper bits of the hash ranges should be
+   // shard1: 11
+   // shard2: 10
+   // shard3: 00
+   // shard4: 01
+
+   String[] highBitsToShard = {"shard3","shard4","shard1","shard2"};
+
+
+   for (int i = 0; i<26; i++) {
+      String id  = new String(Character.toChars('a'+i));
+      int hash = hash(id);
+      System.out.println("hash of " + id + " is " + Integer.toHexString(hash) + " high bits=" + (hash>>>30)
+          + " shard="+highBitsToShard[hash>>>30]);
+    }
+  }
+  ***/
+
+
+
+  DocCollection createCollection(int nSlices, DocRouter router) {
+    List<Range> ranges = router.partitionRange(nSlices, router.fullRange());
+
+    Map<String,Slice> slices = new HashMap<String,Slice>();
+    for (int i=0; i<ranges.size(); i++) {
+      Range range = ranges.get(i);
+      Slice slice = new Slice("shard"+(i+1), null, map("range",range));
+      slices.put(slice.getName(), slice);
+    }
+
+    DocCollection coll = new DocCollection("collection1", slices, null, router);
+    return coll;
+  }
+
+
+
+  // from negative to positive, the upper bits of the hash ranges should be
+  // shard1: top bits:10  80000000:bfffffff
+  // shard2: top bits:11  c0000000:ffffffff
+  // shard3: top bits:00  00000000:3fffffff
+  // shard4: top bits:01  40000000:7fffffff
+
+  /***
+   hash of a is 3c2569b2 high bits=0 shard=shard3
+   hash of b is 95de7e03 high bits=2 shard=shard1
+   hash of c is e132d65f high bits=3 shard=shard2
+   hash of d is 27191473 high bits=0 shard=shard3
+   hash of e is 656c4367 high bits=1 shard=shard4
+   hash of f is 2b64883b high bits=0 shard=shard3
+   hash of g is f18ae416 high bits=3 shard=shard2
+   hash of h is d482b2d3 high bits=3 shard=shard2
+   hash of i is 811a702b high bits=2 shard=shard1
+   hash of j is ca745a39 high bits=3 shard=shard2
+   hash of k is cfbda5d1 high bits=3 shard=shard2
+   hash of l is 1d5d6a2c high bits=0 shard=shard3
+   hash of m is 5ae4385c high bits=1 shard=shard4
+   hash of n is c651d8ac high bits=3 shard=shard2
+   hash of o is 68348473 high bits=1 shard=shard4
+   hash of p is 986fdf9a high bits=2 shard=shard1
+   hash of q is ff8209e8 high bits=3 shard=shard2
+   hash of r is 5c9373f1 high bits=1 shard=shard4
+   hash of s is ff4acaf1 high bits=3 shard=shard2
+   hash of t is ca87df4d high bits=3 shard=shard2
+   hash of u is 62203ae0 high bits=1 shard=shard4
+   hash of v is bdafcc55 high bits=2 shard=shard1
+   hash of w is ff439d1f high bits=3 shard=shard2
+   hash of x is 3e9a9b1b high bits=0 shard=shard3
+   hash of y is 477d9216 high bits=1 shard=shard4
+   hash of z is c1f69a17 high bits=3 shard=shard2
+   ***/
+
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkCLITest.java Fri Jan 18 18:30:54 2013
@@ -94,11 +94,23 @@ public class ZkCLITest extends SolrTestC
     
     assertTrue(zkClient.exists(ZkController.CONFIGS_ZKNODE + "/core0", true));
     assertTrue(zkClient.exists(ZkController.CONFIGS_ZKNODE + "/core1", true));
-    
-
   }
   
   @Test
+  public void testBootstrapWithChroot() throws Exception {
+    String chroot = "/foo/bar";
+    assertFalse(zkClient.exists(chroot, true));
+    
+    String[] args = new String[] {"-zkhost", zkServer.getZkAddress() + chroot,
+        "-cmd", "bootstrap", "-solrhome", ExternalPaths.EXAMPLE_HOME};
+    
+    ZkCLI.main(args);
+    
+    assertTrue(zkClient.exists(chroot + ZkController.CONFIGS_ZKNODE
+        + "/collection1", true));
+  }
+
+  @Test
   public void testMakePath() throws Exception {
     // test bootstrap_conf
     String[] args = new String[] {"-zkhost", zkServer.getZkAddress(), "-cmd",

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkControllerTest.java Fri Jan 18 18:30:54 2013
@@ -31,7 +31,6 @@ import org.apache.solr.core.CoreContaine
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.util.ExternalPaths;
 import org.apache.zookeeper.CreateMode;
-import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -49,6 +48,88 @@ public class ZkControllerTest extends So
     initCore();
   }
 
+  public void testNodeNameUrlConversion() throws Exception {
+
+    // nodeName from parts
+    assertEquals("localhost:8888_solr",
+                 ZkController.generateNodeName("localhost", "8888", "solr"));
+    assertEquals("localhost:8888_solr",
+                 ZkController.generateNodeName("localhost", "8888", "/solr"));
+    assertEquals("localhost:8888_solr",
+                 ZkController.generateNodeName("localhost", "8888", "/solr/"));
+    // root context
+    assertEquals("localhost:8888_", 
+                 ZkController.generateNodeName("localhost", "8888", ""));
+    assertEquals("localhost:8888_", 
+                 ZkController.generateNodeName("localhost", "8888", "/"));
+    // subdir
+    assertEquals("foo-bar:77_solr%2Fsub_dir",
+                 ZkController.generateNodeName("foo-bar", "77", "solr/sub_dir"));
+    assertEquals("foo-bar:77_solr%2Fsub_dir",
+                 ZkController.generateNodeName("foo-bar", "77", "/solr/sub_dir"));
+    assertEquals("foo-bar:77_solr%2Fsub_dir",
+                 ZkController.generateNodeName("foo-bar", "77", "/solr/sub_dir/"));
+
+    // setup a SolrZkClient to do some getBaseUrlForNodeName testing
+    String zkDir = dataDir.getAbsolutePath() + File.separator
+        + "zookeeper/server1/data";
+
+    ZkTestServer server = new ZkTestServer(zkDir);
+    try {
+      server.run();
+
+      AbstractZkTestCase.tryCleanSolrZkNode(server.getZkHost());
+      AbstractZkTestCase.makeSolrZkNode(server.getZkHost());
+
+      SolrZkClient zkClient = new SolrZkClient(server.getZkAddress(), TIMEOUT);
+      try {
+
+        // getBaseUrlForNodeName
+        assertEquals("http://zzz.xxx:1234/solr",
+                     zkClient.getBaseUrlForNodeName("zzz.xxx:1234_solr"));
+        assertEquals("http://xxx:99",
+                     zkClient.getBaseUrlForNodeName("xxx:99_"));
+        assertEquals("http://foo-bar.baz.org:9999/some_dir",
+                     zkClient.getBaseUrlForNodeName("foo-bar.baz.org:9999_some_dir"));
+        assertEquals("http://foo-bar.baz.org:9999/solr/sub_dir",
+                     zkClient.getBaseUrlForNodeName("foo-bar.baz.org:9999_solr%2Fsub_dir"));
+        
+        // generateNodeName + getBaseUrlForNodeName
+        assertEquals("http://foo:9876/solr",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo","9876","solr")));
+        assertEquals("http://foo:9876/solr",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo","9876","/solr")));
+        assertEquals("http://foo:9876/solr",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo","9876","/solr/")));
+        assertEquals("http://foo.bar.com:9876/solr/sub_dir",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo.bar.com","9876","solr/sub_dir")));
+        assertEquals("http://foo.bar.com:9876/solr/sub_dir",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo.bar.com","9876","/solr/sub_dir/")));
+        assertEquals("http://foo-bar:9876",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo-bar","9876","")));
+        assertEquals("http://foo-bar:9876",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo-bar","9876","/")));
+        assertEquals("http://foo-bar.com:80/some_dir",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo-bar.com","80","some_dir")));
+        assertEquals("http://foo-bar.com:80/some_dir",
+                     zkClient.getBaseUrlForNodeName
+                     (ZkController.generateNodeName("foo-bar.com","80","/some_dir")));
+      } finally {
+        zkClient.close();
+      }
+    } finally {
+      server.shutdown();
+    }
+  }
+
   @Test
   public void testReadConfigName() throws Exception {
     String zkDir = dataDir.getAbsolutePath() + File.separator
@@ -82,7 +163,7 @@ public class ZkControllerTest extends So
       cc = getCoreContainer();
       
       ZkController zkController = new ZkController(cc, server.getZkAddress(), TIMEOUT, 10000,
-          "127.0.0.1", "8983", "solr", new CurrentCoreDescriptorProvider() {
+          "127.0.0.1", "8983", "solr", "0", 10000, 10000, new CurrentCoreDescriptorProvider() {
             
             @Override
             public List<CoreDescriptor> getCurrentDescriptors() {
@@ -122,7 +203,7 @@ public class ZkControllerTest extends So
       cc = getCoreContainer();
       
       zkController = new ZkController(cc, server.getZkAddress(),
-          TIMEOUT, 10000, "127.0.0.1", "8983", "solr", new CurrentCoreDescriptorProvider() {
+          TIMEOUT, 10000, "127.0.0.1", "8983", "solr", "0", 10000, 10000, new CurrentCoreDescriptorProvider() {
             
             @Override
             public List<CoreDescriptor> getCurrentDescriptors() {
@@ -143,7 +224,7 @@ public class ZkControllerTest extends So
       }
       testFinished = true;
     } finally {
-      if (!testFinished) {
+      if (!testFinished & zkController != null) {
         zkController.getZkClient().printLayoutToStdOut();
       }
       
@@ -159,7 +240,13 @@ public class ZkControllerTest extends So
   }
 
   private CoreContainer getCoreContainer() {
-    return new CoreContainer(TEMP_DIR.getAbsolutePath());
+    CoreContainer cc = new CoreContainer(TEMP_DIR.getAbsolutePath()) {
+      {
+        initShardHandler(null);
+      }
+    };
+    
+    return cc;
   }
 
   @Override

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/cloud/ZkSolrClientTest.java Fri Jan 18 18:30:54 2013
@@ -209,6 +209,7 @@ public class ZkSolrClientTest extends Ab
 
       zkClient.getChildren("/collections", new Watcher() {
 
+        @Override
         public void process(WatchedEvent event) {
           if (DEBUG) {
             System.out.println("children changed");

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/AlternateDirectoryTest.java Fri Jan 18 18:30:54 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.store.Directory;
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.core.DirectoryFactory.DirContext;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -46,7 +47,7 @@ public class AlternateDirectoryTest exte
     public static volatile Directory dir;
     
     @Override
-    public Directory create(String path) throws IOException {
+    public Directory create(String path, DirContext dirContext) throws IOException {
       openCalled = true;
 
       return dir = newFSDirectory(new File(path));

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/DummyValueSourceParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/DummyValueSourceParser.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/DummyValueSourceParser.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/DummyValueSourceParser.java Fri Jan 18 18:30:54 2013
@@ -19,9 +19,9 @@ package org.apache.solr.core;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.SimpleFloatFunction;
-import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.search.FunctionQParser;
+import org.apache.solr.search.SyntaxError;
 import org.apache.solr.search.ValueSourceParser;
 
 
@@ -38,7 +38,7 @@ public class DummyValueSourceParser exte
   }
 
   @Override
-  public ValueSource parse(FunctionQParser fp) throws ParseException {
+  public ValueSource parse(FunctionQParser fp) throws SyntaxError {
     ValueSource source = fp.parseValueSource();
     ValueSource result = new SimpleFloatFunction(source) {
       @Override

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/FakeDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/FakeDeletionPolicy.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/FakeDeletionPolicy.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/FakeDeletionPolicy.java Fri Jan 18 18:30:54 2013
@@ -32,6 +32,7 @@ public class FakeDeletionPolicy implemen
   private String var2;
 
   //@Override
+  @Override
   public void init(NamedList args) {
     var1 = (String) args.get("var1");
     var2 = (String) args.get("var2");
@@ -46,11 +47,13 @@ public class FakeDeletionPolicy implemen
   }
 
   //  @Override
+  @Override
   public void onCommit(List arg0) throws IOException {
     System.setProperty("onCommit", "test.org.apache.solr.core.FakeDeletionPolicy.onCommit");
   }
 
   //  @Override
+  @Override
   public void onInit(List arg0) throws IOException {
     System.setProperty("onInit", "test.org.apache.solr.core.FakeDeletionPolicy.onInit");
   }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/MockEventListener.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/MockEventListener.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/MockEventListener.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/MockEventListener.java Fri Jan 18 18:30:54 2013
@@ -33,10 +33,12 @@ public class MockEventListener implement
     createCounter.incrementAndGet();
   }
 
+  @Override
   public void init(NamedList args) {
     /* NOOP */
   }
 
+  @Override
   public void postCommit() {
     /* NOOP */
   }
@@ -46,6 +48,7 @@ public class MockEventListener implement
     /* NOOP */
   }
 
+  @Override
   public void newSearcher(SolrIndexSearcher newSearcher, 
                           SolrIndexSearcher currentSearcher) {
     /* NOOP */

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/RAMDirectoryFactoryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/RAMDirectoryFactoryTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/RAMDirectoryFactoryTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/RAMDirectoryFactoryTest.java Fri Jan 18 18:30:54 2013
@@ -22,6 +22,7 @@ import java.io.IOException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.core.DirectoryFactory.DirContext;
 
 /**
  * Test-case for RAMDirectoryFactory
@@ -37,13 +38,13 @@ public class RAMDirectoryFactoryTest ext
     final Directory directory = new RAMDirectory();
     RAMDirectoryFactory factory = new RAMDirectoryFactory()  {
       @Override
-      protected Directory create(String path) {
+      protected Directory create(String path, DirContext dirContext) {
         return directory;
       }
     };
     String path = "/fake/path";
-    Directory dir1 = factory.get(path, null);
-    Directory dir2 = factory.get(path, null);
+    Directory dir1 = factory.get(path, DirContext.DEFAULT, null);
+    Directory dir2 = factory.get(path, DirContext.DEFAULT, null);
     assertEquals("RAMDirectoryFactory should not create new instance of RefCntRamDirectory " +
         "every time open() is called for the same path", dir1, dir2);
 
@@ -53,7 +54,7 @@ public class RAMDirectoryFactoryTest ext
 
   private void dotestOpenSucceedForEmptyDir() throws IOException {
     RAMDirectoryFactory factory = new RAMDirectoryFactory();
-    Directory dir = factory.get("/fake/path", null);
+    Directory dir = factory.get("/fake/path", DirContext.DEFAULT, null);
     assertNotNull("RAMDirectoryFactory should create RefCntRamDirectory even if the path doen't lead " +
         "to index directory on the file system", dir);
     factory.release(dir);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/ResourceLoaderTest.java Fri Jan 18 18:30:54 2013
@@ -17,13 +17,11 @@
 
 package org.apache.solr.core;
 
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
 import junit.framework.Assert;
 
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.analysis.core.KeywordTokenizerFactory;
 import org.apache.lucene.analysis.ngram.NGramFilterFactory;
-import org.apache.solr.SolrIgnoredThreadsFilter;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.handler.admin.LukeRequestHandler;
 import org.apache.solr.handler.component.FacetComponent;
@@ -37,9 +35,6 @@ import java.nio.charset.CharacterCodingE
 import java.util.Arrays;
 import java.util.List;
 
-@ThreadLeakFilters(defaultFilters = true, filters = {
-    SolrIgnoredThreadsFilter.class
-})
 public class ResourceLoaderTest extends LuceneTestCase 
 {
   public void testInstanceDir() throws Exception {

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/SolrCoreCheckLockOnStartupTest.java Fri Jan 18 18:30:54 2013
@@ -32,6 +32,7 @@ import java.io.File;
 
 public class SolrCoreCheckLockOnStartupTest extends SolrTestCaseJ4 {
 
+  @Override
   @Before
   public void setUp() throws Exception {
     super.setUp();

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java Fri Jan 18 18:30:54 2013
@@ -176,6 +176,7 @@ public class SolrCoreTest extends SolrTe
           }
         }
         
+        @Override
         public Integer call() {
           SolrCore core = null;
           int r = 0;
@@ -226,10 +227,10 @@ public class SolrCoreTest extends SolrTe
     Map<String, SolrInfoMBean> infoRegistry = core.getInfoRegistry();
     assertTrue("infoRegistry Size: " + infoRegistry.size() + " is not greater than: " + 0, infoRegistry.size() > 0);
     //try out some that we know are in the config
-    SolrInfoMBean bean = infoRegistry.get(SpellCheckComponent.class.getName());
+    SolrInfoMBean bean = infoRegistry.get(SpellCheckComponent.COMPONENT_NAME);
     assertNotNull("bean not registered", bean);
     //try a default one
-    bean = infoRegistry.get(QueryComponent.class.getName());
+    bean = infoRegistry.get(QueryComponent.COMPONENT_NAME);
     assertNotNull("bean not registered", bean);
     //try a Req Handler, which are stored by name, not clas
     bean = infoRegistry.get("standard");
@@ -243,6 +244,7 @@ public class SolrCoreTest extends SolrTe
 class ClosingRequestHandler extends EmptyRequestHandler implements SolrCoreAware {
   boolean closed = false;
 
+  @Override
   public void inform(SolrCore core) {
     core.addCloseHook( new CloseHook() {
       @Override

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestArbitraryIndexDir.java Fri Jan 18 18:30:54 2013
@@ -29,11 +29,11 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Version;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.handler.SnapPuller;
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.util.TestHarness;
 import org.junit.AfterClass;
@@ -86,12 +86,12 @@ public class TestArbitraryIndexDir exten
   }
 
   @Test
-  public void testLoadNewIndexDir() throws IOException, ParserConfigurationException, SAXException, ParseException {
+  public void testLoadNewIndexDir() throws IOException, ParserConfigurationException, SAXException {
     //add a doc in original index dir
     assertU(adoc("id", String.valueOf(1),
         "name", "name"+String.valueOf(1)));
     //create a new index dir and index.properties file
-    File idxprops = new File(h.getCore().getDataDir() + "index.properties");
+    File idxprops = new File(h.getCore().getDataDir() + SnapPuller.INDEX_PROPERTIES);
     Properties p = new Properties();
     File newDir = new File(h.getCore().getDataDir() + "index_temp");
     newDir.mkdirs();
@@ -102,7 +102,7 @@ public class TestArbitraryIndexDir exten
       p.store(os, "index properties");
     } catch (Exception e) {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
-          "Unable to write index.properties", e);
+          "Unable to write " + SnapPuller.INDEX_PROPERTIES, e);
     } finally {
       if (os != null) os.close();
     }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestCoreContainer.java Fri Jan 18 18:30:54 2013
@@ -110,9 +110,9 @@ public class TestCoreContainer extends S
     cores.persistFile(oneXml);
 
     assertXmlFile(oneXml, "/solr[@persistent='true']",
-        "/solr/cores[@defaultCoreName='collection1' and not(@swappableCacheSize)]",
+        "/solr/cores[@defaultCoreName='collection1' and not(@transientCacheSize)]",
         "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir +
-        "' and @swappable='false' and @loadOnStartup='true' ]", "1=count(/solr/cores/core)");
+        "' and @transient='false' and @loadOnStartup='true' ]", "1=count(/solr/cores/core)");
 
     // create some new cores and sanity check the persistence
     
@@ -142,12 +142,12 @@ public class TestCoreContainer extends S
       assertEquals("cores not added?", 3, cores.getCoreNames().size());
       
       final File twoXml = new File(workDir, "2.solr.xml");
-      cores.swappableCacheSize = 32;
+      cores.transientCacheSize = 32;
 
       cores.persistFile(twoXml);
 
       assertXmlFile(twoXml, "/solr[@persistent='true']",
-          "/solr/cores[@defaultCoreName='collection1' and @swappableCacheSize='32']",
+          "/solr/cores[@defaultCoreName='collection1' and @transientCacheSize='32']",
           "/solr/cores/core[@name='collection1' and @instanceDir='" + instDir
               + "']", "/solr/cores/core[@name='X' and @instanceDir='" + instDir
               + "' and @dataDir='" + dataX + "']",

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestJmxMonitoredMap.java Fri Jan 18 18:30:54 2013
@@ -176,31 +176,38 @@ public class TestJmxMonitoredMap extends
   }
 
   private class MockInfoMBean implements SolrInfoMBean {
+    @Override
     public String getName() {
       return "mock";
     }
 
+    @Override
     public Category getCategory() {
       return Category.OTHER;
     }
 
+    @Override
     public String getDescription() {
       return "mock";
     }
 
+    @Override
     public URL[] getDocs() {
       // TODO Auto-generated method stub
       return null;
     }
 
+    @Override
     public String getVersion() {
       return "mock";
     }
 
+    @Override
     public String getSource() {
       return "mock";
     }
 
+    @Override
     @SuppressWarnings("unchecked")
     public NamedList getStatistics() {
       NamedList myList = new NamedList<Integer>();

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestLazyCores.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestLazyCores.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestLazyCores.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestLazyCores.java Fri Jan 18 18:30:54 2013
@@ -17,6 +17,8 @@ package org.apache.solr.core;
  * limitations under the License.
  */
 
+import org.apache.commons.io.FileUtils;
+import org.apache.lucene.util.IOUtils;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.util.NamedList;
@@ -27,18 +29,15 @@ import org.apache.solr.update.AddUpdateC
 import org.apache.solr.update.CommitUpdateCommand;
 import org.apache.solr.update.UpdateHandler;
 import org.apache.solr.util.RefCounted;
-import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
-import org.xml.sax.SAXException;
 
-import javax.xml.parsers.ParserConfigurationException;
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
-import java.util.Map;
+import java.util.List;
 
 public class TestLazyCores extends SolrTestCaseJ4 {
 
@@ -47,49 +46,76 @@ public class TestLazyCores extends SolrT
     initCore("solrconfig.xml", "schema.xml");
   }
 
-  private CoreContainer cores;
+  private final File _solrHomeDirectory = new File(TEMP_DIR, "org.apache.solr.core.TestLazyCores_testlazy");
 
-  @Before
-  public void before() throws IOException, SAXException, ParserConfigurationException {
-    String solrHome;
-    solrHome = SolrResourceLoader.locateSolrHome();
-    File fconf = new File(solrHome, "solr-lots-of-cores.xml");
+  private static String[] _necessaryConfs = {"schema.xml", "solrconfig.xml", "stopwords.txt", "synonyms.txt",
+      "protwords.txt", "old_synonyms.txt", "currency.xml", "open-exchange-rates.json", "mapping-ISOLatin1Accent.txt"};
+
+  private void copyConfFiles(File home, String subdir) throws IOException {
+
+    File subHome = new File(new File(home, subdir), "conf");
+    assertTrue("Failed to make subdirectory ", subHome.mkdirs());
+    String top = SolrTestCaseJ4.TEST_HOME() + "/collection1/conf";
+    for (String file : _necessaryConfs) {
+      FileUtils.copyFile(new File(top, file), new File(subHome, file));
+    }
+  }
+
+  private CoreContainer init() throws Exception {
+
+    if (_solrHomeDirectory.exists()) {
+      FileUtils.deleteDirectory(_solrHomeDirectory);
+    }
+    assertTrue("Failed to mkdirs workDir", _solrHomeDirectory.mkdirs());
+    for (int idx = 1; idx < 10; ++idx) {
+      copyConfFiles(_solrHomeDirectory, "collection" + idx);
+    }
+
+    File solrXml = new File(_solrHomeDirectory, "solr.xml");
+    FileUtils.write(solrXml, LOTS_SOLR_XML, IOUtils.CHARSET_UTF_8.toString());
+    final CoreContainer cores = new CoreContainer(_solrHomeDirectory.getAbsolutePath());
+    cores.load(_solrHomeDirectory.getAbsolutePath(), solrXml);
+    //  h.getCoreContainer().load(_solrHomeDirectory.getAbsolutePath(), new File(_solrHomeDirectory, "solr.xml"));
 
-    cores = new CoreContainer(solrHome);
-    cores.load(solrHome, fconf);
     cores.setPersistent(false);
+    return cores;
   }
+
+  public void after() throws Exception {
+    if (_solrHomeDirectory.exists()) {
+      FileUtils.deleteDirectory(_solrHomeDirectory);
+    }
+  }
+
   @Test
-  public void testLazyLoad() {
+  public void testLazyLoad() throws Exception {
+    CoreContainer cc = init();
     try {
-      // NOTE: the way this works, this should not assert, however if it's put after the getCore on this collection,
-      // that will cause the core to be loaded and this test will fail.
 
-      Collection<String> names = cores.getCoreNames();
-      for (String name : names) {
-        assertFalse("collectionLazy2".equals(name));
-      }
+      // NOTE: This checks the initial state for loading, no need to do this elsewhere.
+      checkInCores(cc, "collection1", "collectionLazy2", "collectionLazy5");
+      checkNotInCores(cc, "collectionLazy3", "collectionLazy4", "collectionLazy6", "collectionLazy7",
+          "collectionLazy8", "collectionLazy9");
 
-      SolrCore core1 = cores.getCore("collection1");
-      CoreDescriptor cont = core1.getCoreDescriptor();
-      assertFalse("core1 should not be swappable", core1.getCoreDescriptor().isSwappable());
+      SolrCore core1 = cc.getCore("collection1");
+      assertFalse("core1 should not be transient", core1.getCoreDescriptor().isTransient());
       assertTrue("core1 should  be loadable", core1.getCoreDescriptor().isLoadOnStartup());
       assertNotNull(core1.getSolrConfig());
 
-      SolrCore core2 = cores.getCore("collectionLazy2");
-      assertTrue("core2 should not be swappable", core2.getCoreDescriptor().isSwappable());
-      assertFalse("core2 should not be loadable", core2.getCoreDescriptor().isLoadOnStartup());
+      SolrCore core2 = cc.getCore("collectionLazy2");
+      assertTrue("core2 should not be transient", core2.getCoreDescriptor().isTransient());
+      assertTrue("core2 should be loadable", core2.getCoreDescriptor().isLoadOnStartup());
 
-      SolrCore core3 = cores.getCore("collectionLazy3");
-      assertTrue("core3 should not be swappable", core3.getCoreDescriptor().isSwappable());
+      SolrCore core3 = cc.getCore("collectionLazy3");
+      assertTrue("core3 should not be transient", core3.getCoreDescriptor().isTransient());
       assertFalse("core3 should not be loadable", core3.getCoreDescriptor().isLoadOnStartup());
 
-      SolrCore core4 = cores.getCore("collectionLazy4");
-      assertFalse("core4 should not be swappable", core4.getCoreDescriptor().isSwappable());
+      SolrCore core4 = cc.getCore("collectionLazy4");
+      assertFalse("core4 should not be transient", core4.getCoreDescriptor().isTransient());
       assertFalse("core4 should not be loadable", core4.getCoreDescriptor().isLoadOnStartup());
 
-      SolrCore core5 = cores.getCore("collectionLazy5");
-      assertFalse("core5 should not be swappable", core5.getCoreDescriptor().isSwappable());
+      SolrCore core5 = cc.getCore("collectionLazy5");
+      assertFalse("core5 should not be transient", core5.getCoreDescriptor().isTransient());
       assertTrue("core5 should  be loadable", core5.getCoreDescriptor().isLoadOnStartup());
 
       core1.close();
@@ -98,7 +124,7 @@ public class TestLazyCores extends SolrT
       core4.close();
       core5.close();
     } finally {
-      cores.shutdown();
+      cc.shutdown();
     }
   }
 
@@ -106,34 +132,35 @@ public class TestLazyCores extends SolrT
   // will, of course, load it.
   @Test
   public void testLazySearch() throws Exception {
+    CoreContainer cc = init();
     try {
-      // Make sure Lazy2 isn't loaded.
-      checkNotInCores("collectionLazy2");
-      SolrCore core2 = cores.getCore("collectionLazy2");
-
-      addLazy(core2, "id", "0");
-      addLazy(core2, "id", "1", "v_t", "Hello Dude");
-      addLazy(core2, "id", "2", "v_t", "Hello Yonik");
-      addLazy(core2, "id", "3", "v_s", "{!literal}");
-      addLazy(core2, "id", "4", "v_s", "other stuff");
-      addLazy(core2, "id", "5", "v_f", "3.14159");
-      addLazy(core2, "id", "6", "v_f", "8983");
+      // Make sure Lazy4 isn't loaded. Should be loaded on the get
+      checkNotInCores(cc, "collectionLazy4");
+      SolrCore core4 = cc.getCore("collectionLazy4");
+
+      addLazy(core4, "id", "0");
+      addLazy(core4, "id", "1", "v_t", "Hello Dude");
+      addLazy(core4, "id", "2", "v_t", "Hello Yonik");
+      addLazy(core4, "id", "3", "v_s", "{!literal}");
+      addLazy(core4, "id", "4", "v_s", "other stuff");
+      addLazy(core4, "id", "5", "v_f", "3.14159");
+      addLazy(core4, "id", "6", "v_f", "8983");
 
-      SolrQueryRequest req = makeReq(core2);
+      SolrQueryRequest req = makeReq(core4);
       CommitUpdateCommand cmtCmd = new CommitUpdateCommand(req, false);
-      core2.getUpdateHandler().commit(cmtCmd);
+      core4.getUpdateHandler().commit(cmtCmd);
 
-      RefCounted<SolrIndexSearcher> holder = core2.getSearcher();
+      RefCounted<SolrIndexSearcher> holder = core4.getSearcher();
       SolrIndexSearcher searcher = holder.get();
 
       // Just get a couple of searches to work!
       assertQ("test prefix query",
-          makeReq(core2, "q", "{!prefix f=v_t}hel")
+          makeReq(core4, "q", "{!prefix f=v_t}hel")
           , "//result[@numFound='2']"
       );
 
       assertQ("test raw query",
-          makeReq(core2, "q", "{!raw f=v_t}hello")
+          makeReq(core4, "q", "{!raw f=v_t}hello")
           , "//result[@numFound='2']"
       );
 
@@ -146,55 +173,63 @@ public class TestLazyCores extends SolrT
 
       // no analysis is done, so these should match nothing
       assertQ("test raw query",
-          makeReq(core2, "q", "{!raw f=v_t}Hello")
+          makeReq(core4, "q", "{!raw f=v_t}Hello")
           , "//result[@numFound='0']"
       );
       assertQ("test raw query",
-          makeReq(core2, "q", "{!raw f=v_f}1.5")
+          makeReq(core4, "q", "{!raw f=v_f}1.5")
           , "//result[@numFound='0']"
       );
 
-      checkInCores("collectionLazy2");
+      checkInCores(cc, "collectionLazy4");
 
       searcher.close();
-      core2.close();
+      core4.close();
     } finally {
-      cores.shutdown();
+      cc.shutdown();
     }
   }
+
   @Test
-  public void testCachingLimit() {
+  public void testCachingLimit() throws Exception {
+    CoreContainer cc = init();
     try {
-      // NOTE: the way this works, this should not assert, however if it's put after the getCore on this collection,
-      // that will cause the core to be loaded and this test will fail.
-      Collection<String> names = cores.getCoreNames();
+      // First check that all the cores that should be loaded at startup actually are.
 
-      // By putting these in non-alpha order, we're also checking that we're  not just seeing an artifact.
-      SolrCore core1 = cores.getCore("collection1");
-      SolrCore core2 = cores.getCore("collectionLazy3");
-      SolrCore core4 = cores.getCore("collectionLazy4");
-      SolrCore core3 = cores.getCore("collectionLazy2");
-      SolrCore core5 = cores.getCore("collectionLazy5");
+      checkInCores(cc, "collection1",  "collectionLazy2", "collectionLazy5");
+      checkNotInCores(cc,"collectionLazy3", "collectionLazy4", "collectionLazy6",
+          "collectionLazy7", "collectionLazy8", "collectionLazy9");
 
+      // By putting these in non-alpha order, we're also checking that we're  not just seeing an artifact.
+      SolrCore core1 = cc.getCore("collection1");
+      SolrCore core3 = cc.getCore("collectionLazy3");
+      SolrCore core4 = cc.getCore("collectionLazy4");
+      SolrCore core2 = cc.getCore("collectionLazy2");
+      SolrCore core5 = cc.getCore("collectionLazy5");
 
-      checkInCores("collection1", "collectionLazy2", "collectionLazy3", "collectionLazy4", "collectionLazy5");
-      checkNotInCores("collectionLazy6", "collectionLazy7", "collectionLazy8", "collectionLazy9");
+      checkInCores(cc, "collection1", "collectionLazy2", "collectionLazy3", "collectionLazy4", "collectionLazy5");
+      checkNotInCores(cc, "collectionLazy6", "collectionLazy7", "collectionLazy8", "collectionLazy9");
 
       // map should be full up, add one more and verify
-      SolrCore core6 = cores.getCore("collectionLazy6");
-      checkInCores("collection1", "collectionLazy2", "collectionLazy3", "collectionLazy4", "collectionLazy5", "collectionLazy6");
-      checkNotInCores("collectionLazy7", "collectionLazy8", "collectionLazy9");
-
-      SolrCore core7 = cores.getCore("collectionLazy7");
-      checkInCores("collection1", "collectionLazy2", "collectionLazy3", "collectionLazy4", "collectionLazy5", "collectionLazy6", "collectionLazy7");
-      checkNotInCores("collectionLazy8", "collectionLazy9");
-      SolrCore core8 = cores.getCore("collectionLazy8");
-      checkInCores("collection1", "collectionLazy2", "collectionLazy4", "collectionLazy5", "collectionLazy6", "collectionLazy7", "collectionLazy8");
-      checkNotInCores("collectionLazy3", "collectionLazy9");
-
-      SolrCore core9 = cores.getCore("collectionLazy9");
-      checkInCores("collection1", "collectionLazy4", "collectionLazy5", "collectionLazy6", "collectionLazy7", "collectionLazy8", "collectionLazy9");
-      checkNotInCores( "collectionLazy2","collectionLazy3");
+      SolrCore core6 = cc.getCore("collectionLazy6");
+      checkInCores(cc, "collection1", "collectionLazy2", "collectionLazy3", "collectionLazy4", "collectionLazy5",
+          "collectionLazy6");
+      checkNotInCores(cc, "collectionLazy7", "collectionLazy8", "collectionLazy9");
+
+      SolrCore core7 = cc.getCore("collectionLazy7");
+      checkInCores(cc, "collection1", "collectionLazy2", "collectionLazy3", "collectionLazy4", "collectionLazy5",
+          "collectionLazy6", "collectionLazy7");
+      checkNotInCores(cc, "collectionLazy8", "collectionLazy9");
+
+      SolrCore core8 = cc.getCore("collectionLazy8");
+      checkInCores(cc, "collection1", "collectionLazy2", "collectionLazy4", "collectionLazy5", "collectionLazy6",
+          "collectionLazy7", "collectionLazy8");
+      checkNotInCores(cc, "collectionLazy3", "collectionLazy9");
+
+      SolrCore core9 = cc.getCore("collectionLazy9");
+      checkInCores(cc, "collection1", "collectionLazy4", "collectionLazy5", "collectionLazy6", "collectionLazy7",
+          "collectionLazy8", "collectionLazy9");
+      checkNotInCores(cc, "collectionLazy2", "collectionLazy3");
 
 
       // Note decrementing the count when the core is removed from the lazyCores list is appropriate, since the
@@ -209,19 +244,57 @@ public class TestLazyCores extends SolrT
       core8.close();
       core9.close();
     } finally {
-      cores.shutdown();
+      cc.shutdown();
     }
   }
 
-  private void checkNotInCores(String... nameCheck) {
-    Collection<String> names = cores.getCoreNames();
+  // Test case for SOLR-4300
+  @Test
+  public void testRace() throws Exception {
+    final List<SolrCore> _theCores = new ArrayList<SolrCore>();
+    final CoreContainer cc = init();
+    try {
+
+      Thread[] threads = new Thread[15];
+      for (int idx = 0; idx < threads.length; idx++) {
+        threads[idx] = new Thread() {
+          @Override
+          public void run() {
+            SolrCore core = cc.getCore("collectionLazy3");
+            synchronized (_theCores) {
+              _theCores.add(core);
+            }
+          }
+        };
+        threads[idx].start();
+      }
+
+      for (Thread thread : threads) {
+        thread.join();
+      }
+
+      for (int idx = 0; idx < _theCores.size() - 1; ++idx) {
+        assertEquals("Cores should be the same!", _theCores.get(idx), _theCores.get(idx + 1));
+      }
+
+      for (SolrCore core : _theCores) {
+        core.close();
+      }
+
+    } finally {
+      cc.shutdown();
+    }
+  }
+
+  private void checkNotInCores(CoreContainer cc, String... nameCheck) {
+    Collection<String> names = cc.getCoreNames();
     for (String name : nameCheck) {
       assertFalse("core " + name + " was found in the list of cores", names.contains(name));
     }
   }
 
-  private void checkInCores(String... nameCheck) {
-    Collection<String> names = cores.getCoreNames();
+  private void checkInCores(CoreContainer cc, String... nameCheck) {
+    Collection<String> names = cc.getCoreNames();
     for (String name : nameCheck) {
       assertTrue("core " + name + " was not found in the list of cores", names.contains(name));
     }
@@ -251,10 +324,24 @@ public class TestLazyCores extends SolrT
     if (q.length % 2 != 0) {
       throw new RuntimeException("The length of the string array (query arguments) needs to be even");
     }
-    Map.Entry<String, String>[] entries = new NamedList.NamedListEntry[q.length / 2];
+    NamedList.NamedListEntry[] entries = new NamedList.NamedListEntry[q.length / 2];
     for (int i = 0; i < q.length; i += 2) {
       entries[i / 2] = new NamedList.NamedListEntry<String>(q[i], q[i + 1]);
     }
-    return new LocalSolrQueryRequest(core, new NamedList(entries));
+    return new LocalSolrQueryRequest(core, new NamedList<Object>(entries));
   }
+
+  private final static String LOTS_SOLR_XML = " <solr persistent=\"false\"> " +
+      "<cores adminPath=\"/admin/cores\" defaultCoreName=\"collectionLazy2\" transientCacheSize=\"4\">  " +
+      "<core name=\"collection1\" instanceDir=\"collection1\" /> " +
+      "<core name=\"collectionLazy2\" instanceDir=\"collection2\" transient=\"true\" loadOnStartup=\"true\"  /> " +
+      "<core name=\"collectionLazy3\" instanceDir=\"collection3\" transient=\"on\" loadOnStartup=\"false\"/> " +
+      "<core name=\"collectionLazy4\" instanceDir=\"collection4\" transient=\"false\" loadOnStartup=\"false\"/> " +
+      "<core name=\"collectionLazy5\" instanceDir=\"collection5\" transient=\"false\" loadOnStartup=\"true\"/> " +
+      "<core name=\"collectionLazy6\" instanceDir=\"collection6\" transient=\"true\" loadOnStartup=\"false\" /> " +
+      "<core name=\"collectionLazy7\" instanceDir=\"collection7\" transient=\"true\" loadOnStartup=\"false\" /> " +
+      "<core name=\"collectionLazy8\" instanceDir=\"collection8\" transient=\"true\" loadOnStartup=\"false\" /> " +
+      "<core name=\"collectionLazy9\" instanceDir=\"collection9\" transient=\"true\" loadOnStartup=\"false\" /> " +
+      "</cores> " +
+      "</solr>";
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestSolrXMLSerializer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestSolrXMLSerializer.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestSolrXMLSerializer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/core/TestSolrXMLSerializer.java Fri Jan 18 18:30:54 2013
@@ -82,7 +82,7 @@ public class TestSolrXMLSerializer exten
     assertResults(((StringWriter) w).getBuffer().toString().getBytes("UTF-8"));
     
     // again with default file
-    File tmpFile = _TestUtil.getTempDir("solr.xml");
+    File tmpFile = _TestUtil.createTempFile("solr.xml", null, TEMP_DIR);
     
     serializer.persistFile(tmpFile, solrXMLDef);
 

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/DocumentAnalysisRequestHandlerTest.java Fri Jan 18 18:30:54 2013
@@ -122,6 +122,7 @@ public class DocumentAnalysisRequestHand
       sourceInfo = "rawBytes";
     }
 
+    @Override
     public InputStream getStream() throws IOException {
       return new ByteArrayInputStream(bytes);
     }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/TestReplicationHandler.java Fri Jan 18 18:30:54 2013
@@ -111,6 +111,7 @@ public class TestReplicationHandler exte
     }
   }
 
+  @Override
   @After
   public void tearDown() throws Exception {
     super.tearDown();
@@ -137,6 +138,8 @@ public class TestReplicationHandler exte
       // setup the server...
       String url = "http://127.0.0.1:" + port + context;
       HttpSolrServer s = new HttpSolrServer(url);
+      s.setConnectionTimeout(15000);
+      s.setSoTimeout(60000);
       s.setDefaultMaxConnectionsPerHost(100);
       s.setMaxTotalConnections(100);
       return s;
@@ -177,7 +180,7 @@ public class TestReplicationHandler exte
       docList = (SolrDocumentList) res.get("response");
       timeSlept += 100;
       Thread.sleep(100);
-    } while(docList.getNumFound() != expectedDocCount && timeSlept < 30000);
+    } while(docList.getNumFound() != expectedDocCount && timeSlept < 45000);
     return res;
   }
   
@@ -656,6 +659,8 @@ public class TestReplicationHandler exte
   }
 
   private void doTestReplicateAfterCoreReload() throws Exception {
+    int docs = TEST_NIGHTLY ? 200000 : 0;
+    
     //stop slave
     slaveJetty.stop();
 
@@ -670,14 +675,14 @@ public class TestReplicationHandler exte
     masterClient = createNewSolrServer(masterJetty.getLocalPort());
 
     masterClient.deleteByQuery("*:*");
-    for (int i = 0; i < 10; i++)
+    for (int i = 0; i < docs; i++)
       index(masterClient, "id", i, "name", "name = " + i);
 
     masterClient.commit();
 
-    NamedList masterQueryRsp = rQuery(10, "*:*", masterClient);
+    NamedList masterQueryRsp = rQuery(docs, "*:*", masterClient);
     SolrDocumentList masterQueryResult = (SolrDocumentList) masterQueryRsp.get("response");
-    assertEquals(10, masterQueryResult.getNumFound());
+    assertEquals(docs, masterQueryResult.getNumFound());
     
     slave.setTestPort(masterJetty.getLocalPort());
     slave.copyConfigFile(slave.getSolrConfigFile(), "solrconfig.xml");
@@ -687,9 +692,9 @@ public class TestReplicationHandler exte
     slaveClient = createNewSolrServer(slaveJetty.getLocalPort());
     
     //get docs from slave and check if number is equal to master
-    NamedList slaveQueryRsp = rQuery(10, "*:*", slaveClient);
+    NamedList slaveQueryRsp = rQuery(docs, "*:*", slaveClient);
     SolrDocumentList slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(10, slaveQueryResult.getNumFound());
+    assertEquals(docs, slaveQueryResult.getNumFound());
     
     //compare results
     String cmp = BaseDistributedSearchTestCase.compare(masterQueryResult, slaveQueryResult, 0, null);
@@ -703,19 +708,19 @@ public class TestReplicationHandler exte
     assertEquals(version, getIndexVersion(masterClient).get("indexversion"));
     assertEquals(commits.get("commits"), getCommits(masterClient).get("commits"));
     
-    index(masterClient, "id", 110, "name", "name = 1");
-    index(masterClient, "id", 120, "name", "name = 2");
+    index(masterClient, "id", docs + 10, "name", "name = 1");
+    index(masterClient, "id", docs + 20, "name", "name = 2");
 
     masterClient.commit();
     
-    NamedList resp =  rQuery(12, "*:*", masterClient);
+    NamedList resp =  rQuery(docs + 2, "*:*", masterClient);
     masterQueryResult = (SolrDocumentList) resp.get("response");
-    assertEquals(12, masterQueryResult.getNumFound());
+    assertEquals(docs + 2, masterQueryResult.getNumFound());
     
     //get docs from slave and check if number is equal to master
-    slaveQueryRsp = rQuery(12, "*:*", slaveClient);
+    slaveQueryRsp = rQuery(docs + 2, "*:*", slaveClient);
     slaveQueryResult = (SolrDocumentList) slaveQueryRsp.get("response");
-    assertEquals(12, slaveQueryResult.getNumFound());
+    assertEquals(docs + 2, slaveQueryResult.getNumFound());
     
     // NOTE: revert config on master.
     master.copyConfigFile(CONF_DIR + "solrconfig-master.xml", "solrconfig.xml");
@@ -933,6 +938,7 @@ public class TestReplicationHandler exte
   
       File[] files = dataDir.listFiles(new FilenameFilter() {
         
+          @Override
           public boolean accept(File dir, String name) {
             if(name.startsWith("snapshot")) {
               return true;

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/admin/ShowFileRequestHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/admin/ShowFileRequestHandlerTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/admin/ShowFileRequestHandlerTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/admin/ShowFileRequestHandlerTest.java Fri Jan 18 18:30:54 2013
@@ -24,7 +24,10 @@ import org.apache.solr.client.solrj.Solr
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.util.ExternalPaths;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.response.SolrQueryResponse;
 import org.junit.BeforeClass;
 
 import java.io.IOException;
@@ -43,6 +46,41 @@ public class ShowFileRequestHandlerTest 
     createJetty(ExternalPaths.EXAMPLE_HOME, null, null);
   }
 
+  public void test404ViaHttp() throws SolrServerException {
+    SolrServer server = getSolrServer();
+    QueryRequest request = new QueryRequest(params("file",
+                                                   "does-not-exist-404.txt"));
+    request.setPath("/admin/file");
+    try {
+      QueryResponse resp = request.process(server);
+      fail("didn't get 404 exception");
+    } catch (SolrException e) {
+      assertEquals(404, e.code());
+    }
+  }
+
+  public void test404Locally() throws Exception {
+
+    // we need to test that executing the handler directly does not 
+    // throw an exception, just sets the exception on the response.
+    initCore("solrconfig.xml", "schema.xml");
+    try {
+      // bypass TestHarness since it will throw any exception found in the
+      // response.
+      SolrCore core = h.getCore();
+      SolrQueryResponse rsp = new SolrQueryResponse();
+      core.execute(core.getRequestHandler("/admin/file"),
+                   req("file", "does-not-exist-404.txt"), rsp);
+      assertNotNull("no exception in response", rsp.getException());
+      assertTrue("wrong type of exception: " + rsp.getException().getClass(),
+                 rsp.getException() instanceof SolrException);
+      assertEquals(404, ((SolrException)rsp.getException()).code());
+
+    } catch (Exception e) {
+      assertNull("Should not have caught an exception", e);
+    }
+  }
+
   public void testDirList() throws SolrServerException {
     SolrServer server = getSolrServer();
     //assertQ(req("qt", "/admin/file")); TODO file bug that SolrJettyTestBase extends SolrTestCaseJ4

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/component/DistributedQueryElevationComponentTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/component/DistributedQueryElevationComponentTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/component/DistributedQueryElevationComponentTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/component/DistributedQueryElevationComponentTest.java Fri Jan 18 18:30:54 2013
@@ -87,6 +87,7 @@ public class DistributedQueryElevationCo
     query("q", "ZZZZ", "qt", "/elevate", "shards.qt", "/elevate", "rows", "500", CommonParams.FL, "*, [elevated]", "forceElevation", "true", "sort", "id desc");
   }
   
+  @Override
   protected void indexr(Object... fields) throws Exception {
     SolrInputDocument doc = new SolrInputDocument();
     addFields(doc, fields);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/component/TermVectorComponentDistributedTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/component/TermVectorComponentDistributedTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/component/TermVectorComponentDistributedTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/handler/component/TermVectorComponentDistributedTest.java Fri Jan 18 18:30:54 2013
@@ -22,6 +22,7 @@ import org.apache.solr.common.params.Ter
 
 public class TermVectorComponentDistributedTest extends BaseDistributedSearchTestCase {
 
+  @Override
   public void doTest() throws Exception {
 
     handle.clear();

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/request/JSONWriterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/request/JSONWriterTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/request/JSONWriterTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/request/JSONWriterTest.java Fri Jan 18 18:30:54 2013
@@ -34,6 +34,7 @@ import org.apache.solr.response.PythonRe
 import org.apache.solr.response.QueryResponseWriter;
 import org.apache.solr.response.RubyResponseWriter;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.SolrReturnFields;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -105,7 +106,7 @@ public class JSONWriterTest extends Solr
     SolrQueryResponse rsp = new SolrQueryResponse();
     JSONResponseWriter w = new JSONResponseWriter();
 
-    ReturnFields returnFields = new ReturnFields(req);
+    ReturnFields returnFields = new SolrReturnFields(req);
     rsp.setReturnFields(returnFields);
 
     StringWriter buf = new StringWriter();

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/request/TestBinaryResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/request/TestBinaryResponseWriter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/request/TestBinaryResponseWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/request/TestBinaryResponseWriter.java Fri Jan 18 18:30:54 2013
@@ -30,6 +30,7 @@ import org.apache.solr.response.BinaryQu
 import org.apache.solr.response.BinaryResponseWriter.Resolver;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.ReturnFields;
+import org.apache.solr.search.SolrReturnFields;
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.junit.BeforeClass;
 
@@ -81,7 +82,7 @@ public class TestBinaryResponseWriter ex
     in.addField("ddd_s", "ddd");
     in.addField("eee_s", "eee");    
 
-    Resolver r = new Resolver(req, new ReturnFields(req));
+    Resolver r = new Resolver(req, new SolrReturnFields(req));
     Object o = r.resolve(in, new JavaBinCodec());
 
     assertNotNull("obj is null", o);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/response/TestCSVResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/response/TestCSVResponseWriter.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/response/TestCSVResponseWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/response/TestCSVResponseWriter.java Fri Jan 18 18:30:54 2013
@@ -23,6 +23,7 @@ import org.apache.solr.common.SolrDocume
 import org.apache.solr.common.util.DateUtil;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.ReturnFields;
+import org.apache.solr.search.SolrReturnFields;
 import org.junit.*;
 
 import java.io.StringWriter;
@@ -145,19 +146,19 @@ public class TestCSVResponseWriter exten
     rsp.add("response", sdl);
     QueryResponseWriter w = new CSVResponseWriter();
     
-    rsp.setReturnFields( new ReturnFields("id,foo_s", req) );
+    rsp.setReturnFields( new SolrReturnFields("id,foo_s", req) );
     StringWriter buf = new StringWriter();
     w.write(buf, req, rsp);
     assertEquals("id,foo_s\n1,hi\n2,\n", buf.toString());
 
     // try scores
-    rsp.setReturnFields( new ReturnFields("id,score,foo_s", req) );
+    rsp.setReturnFields( new SolrReturnFields("id,score,foo_s", req) );
     buf = new StringWriter();
     w.write(buf, req, rsp);
     assertEquals("id,score,foo_s\n1,2.718,hi\n2,89.83,\n", buf.toString());
 
     // get field values from docs... should be ordered and not include score unless requested
-    rsp.setReturnFields( new ReturnFields("*", req) );
+    rsp.setReturnFields( new SolrReturnFields("*", req) );
     buf = new StringWriter();
     w.write(buf, req, rsp);
     assertEquals("id,foo_i,foo_s,foo_l,foo_b,foo_f,foo_d,foo_dt,v_ss,v2_ss\n" +
@@ -167,14 +168,14 @@ public class TestCSVResponseWriter exten
     
 
     // get field values and scores - just check that the scores are there... we don't guarantee where
-    rsp.setReturnFields( new ReturnFields("*,score", req) );
+    rsp.setReturnFields( new SolrReturnFields("*,score", req) );
     buf = new StringWriter();
     w.write(buf, req, rsp);
     String s = buf.toString();
     assertTrue(s.indexOf("score") >=0 && s.indexOf("2.718") > 0 && s.indexOf("89.83") > 0 );
     
     // Test field globs
-    rsp.setReturnFields( new ReturnFields("id,foo*", req) );
+    rsp.setReturnFields( new SolrReturnFields("id,foo*", req) );
     buf = new StringWriter();
     w.write(buf, req, rsp);
     assertEquals("id,foo_i,foo_s,foo_l,foo_b,foo_f,foo_d,foo_dt\n" +
@@ -182,7 +183,7 @@ public class TestCSVResponseWriter exten
         "2,,,,,,,\n",
       buf.toString());
 
-    rsp.setReturnFields( new ReturnFields("id,*_d*", req) );
+    rsp.setReturnFields( new SolrReturnFields("id,*_d*", req) );
     buf = new StringWriter();
     w.write(buf, req, rsp);
     assertEquals("id,foo_d,foo_dt\n" +

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/OpenExchangeRatesOrgProviderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/OpenExchangeRatesOrgProviderTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/OpenExchangeRatesOrgProviderTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/OpenExchangeRatesOrgProviderTest.java Fri Jan 18 18:30:54 2013
@@ -35,6 +35,7 @@ public class OpenExchangeRatesOrgProvide
   private final Map<String,String> emptyParams = new HashMap<String,String>();
   private Map<String,String> mockParams;
 
+  @Override
   @Before
   public void setUp() throws Exception {
     super.setUp();

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/PreAnalyzedFieldTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/PreAnalyzedFieldTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/PreAnalyzedFieldTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/PreAnalyzedFieldTest.java Fri Jan 18 18:30:54 2013
@@ -70,6 +70,7 @@ public class PreAnalyzedFieldTest extend
   int props = 
     FieldProperties.INDEXED | FieldProperties.STORED;
   
+  @Override
   public void setUp() throws Exception {
     super.setUp();
     field = new SchemaField("content", new TextField(), props, null);

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/TestBinaryField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/TestBinaryField.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/TestBinaryField.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/schema/TestBinaryField.java Fri Jan 18 18:30:54 2013
@@ -16,32 +16,26 @@
  */
 package org.apache.solr.schema;
 
-import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters;
-import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
+import java.io.File;
+import java.io.FileOutputStream;
+import java.nio.ByteBuffer;
+import java.util.List;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.SolrIgnoredThreadsFilter;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.beans.Field;
 import org.apache.solr.client.solrj.embedded.JettySolrRunner;
 import org.apache.solr.client.solrj.impl.HttpSolrServer;
 import org.apache.solr.client.solrj.response.QueryResponse;
-import org.apache.solr.common.SolrDocument;
-import org.apache.solr.common.SolrDocumentList;
-import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.*;
 import org.apache.solr.core.SolrResourceLoader;
 import org.junit.Rule;
 import org.junit.rules.RuleChain;
 import org.junit.rules.TestRule;
 
-import java.io.File;
-import java.io.FileOutputStream;
-import java.nio.ByteBuffer;
-import java.util.List;
+import com.carrotsearch.randomizedtesting.rules.SystemPropertiesRestoreRule;
 
-@ThreadLeakFilters(defaultFilters = true, filters = {
-    SolrIgnoredThreadsFilter.class
-})
 public class TestBinaryField extends LuceneTestCase {
   HttpSolrServer server;
   JettySolrRunner jetty;

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/search/FooQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/search/FooQParserPlugin.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/search/FooQParserPlugin.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/search/FooQParserPlugin.java Fri Jan 18 18:30:54 2013
@@ -17,7 +17,6 @@
 
 package org.apache.solr.search;
 
-import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
@@ -32,6 +31,7 @@ public class FooQParserPlugin extends QP
     return new FooQParser(qstr, localParams, params, req);
   }
 
+  @Override
   public void init(NamedList args) {
   }
 }
@@ -42,7 +42,7 @@ class FooQParser extends QParser {
   }
 
   @Override
-  public Query parse() throws ParseException {
+  public Query parse() throws SyntaxError {
     return new TermQuery(new Term(localParams.get(QueryParsing.F), localParams.get(QueryParsing.V)));
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/search/ReturnFieldsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/search/ReturnFieldsTest.java?rev=1435287&r1=1435286&r2=1435287&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/search/ReturnFieldsTest.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/core/src/test/org/apache/solr/search/ReturnFieldsTest.java Fri Jan 18 18:30:54 2013
@@ -97,7 +97,7 @@ public class ReturnFieldsTest extends So
 
   @Test
   public void testSeparators() {
-    ReturnFields rf = new ReturnFields( req("fl", "id name test subject score") );
+    ReturnFields rf = new SolrReturnFields( req("fl", "id name test subject score") );
     assertTrue( rf.wantsScore() );
     assertTrue( rf.wantsField( "id" ) );
     assertTrue( rf.wantsField( "name" ) );
@@ -108,7 +108,7 @@ public class ReturnFieldsTest extends So
     assertFalse( rf.wantsField( "xxx" ) );
     assertTrue( rf.getTransformer() instanceof ScoreAugmenter);
 
-    rf = new ReturnFields( req("fl", "id,name,test,subject,score") );
+    rf = new SolrReturnFields( req("fl", "id,name,test,subject,score") );
     assertTrue( rf.wantsScore() );
     assertTrue( rf.wantsField( "id" ) );
     assertTrue( rf.wantsField( "name" ) );
@@ -119,7 +119,7 @@ public class ReturnFieldsTest extends So
     assertFalse( rf.wantsField( "xxx" ) );
     assertTrue( rf.getTransformer() instanceof ScoreAugmenter);
 
-    rf = new ReturnFields( req("fl", "id,name test,subject score") );
+    rf = new SolrReturnFields( req("fl", "id,name test,subject score") );
     assertTrue( rf.wantsScore() );
     assertTrue( rf.wantsField( "id" ) );
     assertTrue( rf.wantsField( "name" ) );
@@ -130,7 +130,7 @@ public class ReturnFieldsTest extends So
     assertFalse( rf.wantsField( "xxx" ) );
     assertTrue( rf.getTransformer() instanceof ScoreAugmenter);
 
-    rf = new ReturnFields( req("fl", "id, name  test , subject,score") );
+    rf = new SolrReturnFields( req("fl", "id, name  test , subject,score") );
     assertTrue( rf.wantsScore() );
     assertTrue( rf.wantsField( "id" ) );
     assertTrue( rf.wantsField( "name" ) );
@@ -144,20 +144,20 @@ public class ReturnFieldsTest extends So
 
   @Test
   public void testWilcards() {
-    ReturnFields rf = new ReturnFields( req("fl", "*") );
+    ReturnFields rf = new SolrReturnFields( req("fl", "*") );
     assertFalse( rf.wantsScore() );
     assertTrue( rf.wantsField( "xxx" ) );
     assertTrue( rf.wantsAllFields() );
     assertNull( rf.getTransformer() );
 
-    rf = new ReturnFields( req("fl", " * ") );
+    rf = new SolrReturnFields( req("fl", " * ") );
     assertFalse( rf.wantsScore() );
     assertTrue( rf.wantsField( "xxx" ) );
     assertTrue( rf.wantsAllFields() );
     assertNull( rf.getTransformer() );
 
     // Check that we want wildcards
-    rf = new ReturnFields( req("fl", "id,aaa*,*bbb") );
+    rf = new SolrReturnFields( req("fl", "id,aaa*,*bbb") );
     assertTrue( rf.wantsField( "id" ) );
     assertTrue( rf.wantsField( "aaaxxx" ) );
     assertFalse(rf.wantsField("xxxaaa"));
@@ -169,7 +169,7 @@ public class ReturnFieldsTest extends So
 
   @Test
   public void testManyParameters() {
-    ReturnFields rf = new ReturnFields( req("fl", "id name", "fl", "test subject", "fl", "score") );
+    ReturnFields rf = new SolrReturnFields( req("fl", "id name", "fl", "test subject", "fl", "score") );
     assertTrue( rf.wantsScore() );
     assertTrue( rf.wantsField( "id" ) );
     assertTrue( rf.wantsField( "name" ) );
@@ -183,7 +183,7 @@ public class ReturnFieldsTest extends So
 
   @Test
   public void testFunctions() {
-    ReturnFields rf = new ReturnFields( req("fl", "id sum(1,1)") );
+    ReturnFields rf = new SolrReturnFields( req("fl", "id sum(1,1)") );
     assertFalse(rf.wantsScore());
     assertTrue( rf.wantsField( "id" ) );
     assertFalse( rf.wantsAllFields() );
@@ -194,41 +194,41 @@ public class ReturnFieldsTest extends So
 
   @Test
   public void testTransformers() {
-    ReturnFields rf = new ReturnFields( req("fl", "[explain]") );
+    ReturnFields rf = new SolrReturnFields( req("fl", "[explain]") );
     assertFalse( rf.wantsScore() );
     assertFalse(rf.wantsField("id"));
     assertFalse(rf.wantsAllFields());
     assertEquals( "[explain]", rf.getTransformer().getName() );
 
-    rf = new ReturnFields( req("fl", "[shard],id") );
+    rf = new SolrReturnFields( req("fl", "[shard],id") );
     assertFalse( rf.wantsScore() );
     assertTrue(rf.wantsField("id"));
     assertFalse(rf.wantsField("xxx"));
     assertFalse(rf.wantsAllFields());
     assertEquals( "[shard]", rf.getTransformer().getName() );
 
-    rf = new ReturnFields( req("fl", "[docid]") );
+    rf = new SolrReturnFields( req("fl", "[docid]") );
     assertFalse( rf.wantsScore() );
     assertFalse( rf.wantsField( "id" ) );
     assertFalse(rf.wantsField("xxx"));
     assertFalse(rf.wantsAllFields());
     assertEquals( "[docid]", rf.getTransformer().getName() );
 
-    rf = new ReturnFields( req("fl", "mydocid:[docid]") );
+    rf = new SolrReturnFields( req("fl", "mydocid:[docid]") );
     assertFalse( rf.wantsScore() );
     assertFalse( rf.wantsField( "id" ) );
     assertFalse(rf.wantsField("xxx"));
     assertFalse(rf.wantsAllFields());
     assertEquals( "mydocid", rf.getTransformer().getName() );
 
-    rf = new ReturnFields( req("fl", "[docid][shard]") );
+    rf = new SolrReturnFields( req("fl", "[docid][shard]") );
     assertFalse( rf.wantsScore() );
     assertFalse(rf.wantsField("xxx"));
     assertFalse(rf.wantsAllFields());
     assertTrue( rf.getTransformer() instanceof DocTransformers);
     assertEquals(2, ((DocTransformers)rf.getTransformer()).size());
 
-    rf = new ReturnFields( req("fl", "[xxxxx]") );
+    rf = new SolrReturnFields( req("fl", "[xxxxx]") );
     assertFalse( rf.wantsScore() );
     assertFalse( rf.wantsField( "id" ) );
     assertFalse(rf.wantsAllFields());
@@ -237,7 +237,7 @@ public class ReturnFieldsTest extends So
 
   @Test
   public void testAliases() {
-    ReturnFields rf = new ReturnFields( req("fl", "newId:id newName:name newTest:test newSubject:subject") );
+    ReturnFields rf = new SolrReturnFields( req("fl", "newId:id newName:name newTest:test newSubject:subject") );
     assertTrue(rf.wantsField("id"));
     assertTrue(rf.wantsField("name"));
     assertTrue(rf.wantsField("test"));
@@ -249,7 +249,7 @@ public class ReturnFieldsTest extends So
     assertFalse(rf.wantsField("xxx"));
     assertFalse(rf.wantsAllFields());
 
-    rf = new ReturnFields( req("fl", "newId:id newName:name newTest:test newSubject:subject score") );
+    rf = new SolrReturnFields( req("fl", "newId:id newName:name newTest:test newSubject:subject score") );
     assertTrue(rf.wantsField("id"));
     assertTrue(rf.wantsField("name"));
     assertTrue(rf.wantsField("test"));
@@ -268,7 +268,7 @@ public class ReturnFieldsTest extends So
   // the simplest case of fl=foo-bar to work
   @Test
   public void testHyphenInFieldName() {
-    ReturnFields rf = new ReturnFields(req("fl", "id-test"));
+    ReturnFields rf = new SolrReturnFields(req("fl", "id-test"));
     assertFalse(rf.wantsScore());
     assertTrue(rf.wantsField("id-test"));
     assertFalse(rf.wantsField("xxx"));
@@ -277,20 +277,20 @@ public class ReturnFieldsTest extends So
 
   @Test
   public void testTrailingDotInFieldName() {
-    ReturnFields rf = new ReturnFields(req("fl", "id.test"));
+    ReturnFields rf = new SolrReturnFields(req("fl", "id.test"));
     assertFalse(rf.wantsScore());
     assertTrue(rf.wantsField("id.test"));
     assertFalse(rf.wantsField("xxx"));
     assertFalse(rf.wantsAllFields());
 
-    rf = new ReturnFields(req("fl", "test:id.test"));
+    rf = new SolrReturnFields(req("fl", "test:id.test"));
     assertFalse(rf.wantsScore());
     assertTrue(rf.wantsField("id.test"));
     assertTrue(rf.wantsField("test"));
     assertFalse(rf.wantsField("xxx"));
     assertFalse(rf.wantsAllFields());
 
-    rf = new ReturnFields(req("fl", "test.id:id.test"));
+    rf = new SolrReturnFields(req("fl", "test.id:id.test"));
     assertFalse(rf.wantsScore());
     assertTrue(rf.wantsField("id.test"));
     assertTrue(rf.wantsField("test.id"));
@@ -300,7 +300,7 @@ public class ReturnFieldsTest extends So
 
   @Test
   public void testTrailingDollarInFieldName() {
-    ReturnFields rf = new ReturnFields(req("fl", "id$test"));
+    ReturnFields rf = new SolrReturnFields(req("fl", "id$test"));
     assertFalse(rf.wantsScore());
     assertTrue(rf.wantsField("id$test"));
     assertFalse(rf.wantsField("xxx"));
@@ -325,7 +325,7 @@ public class ReturnFieldsTest extends So
         _TestUtil.randomWhitespace(r, 0, 3);
 
       final String fl = id + (r.nextBoolean() ? "" : ",") + foo_i;
-      ReturnFields rf = new ReturnFields(req("fl", fl));
+      ReturnFields rf = new SolrReturnFields(req("fl", fl));
 
       assertFalse("score ("+fl+")", rf.wantsScore());