You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2014/06/05 15:22:10 UTC

svn commit: r1600647 - in /lucene/dev/branches/LUCENE-5716/lucene: ./ core/src/java/org/apache/lucene/search/ test-framework/ test-framework/src/aspects/ test-framework/src/aspects/META-INF/ test-framework/src/aspects/org/ test-framework/src/aspects/or...

Author: dweiss
Date: Thu Jun  5 13:22:09 2014
New Revision: 1600647

URL: http://svn.apache.org/r1600647
Log:
A no-commit prototype of the test sandbox running/ instrumented with aspectj aspects.

Added:
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/META-INF/
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/META-INF/aop-ajc.xml   (with props)
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/apache/
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/apache/lucene/
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/apache/lucene/util/
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/apache/lucene/util/NoDescriptorLeaksAspect.aj
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAspectJAspects.java   (with props)
Modified:
    lucene/dev/branches/LUCENE-5716/lucene/common-build.xml
    lucene/dev/branches/LUCENE-5716/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
    lucene/dev/branches/LUCENE-5716/lucene/ivy-versions.properties
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/build.xml
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/ivy.xml
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/AbstractBeforeAfterRule.java
    lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java

Modified: lucene/dev/branches/LUCENE-5716/lucene/common-build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/common-build.xml?rev=1600647&r1=1600646&r2=1600647&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/common-build.xml (original)
+++ lucene/dev/branches/LUCENE-5716/lucene/common-build.xml Thu Jun  5 13:22:09 2014
@@ -921,6 +921,7 @@
             <!-- Classpaths. -->
             <classpath refid="@{junit.classpath}"/>
             <classpath refid="clover.classpath" />
+        	<classpath location="${common.dir}/build/test-framework/aspects.jar" />
 
             <!-- Assertions. -->
             <assertions>
@@ -928,6 +929,8 @@
               <enable package="org.apache.solr"/>
             </assertions>  
 
+            <jvmarg line="-javaagent:${common.dir}/test-framework/lib/aspectjweaver-1.8.0.jar" />
+
             <!-- JVM arguments and system properties. -->
             <jvmarg line="${args}"/>
             <jvmarg line="${tests.heapdump.args}"/>

Modified: lucene/dev/branches/LUCENE-5716/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java?rev=1600647&r1=1600646&r2=1600647&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java (original)
+++ lucene/dev/branches/LUCENE-5716/lucene/core/src/java/org/apache/lucene/search/TopFieldCollector.java Thu Jun  5 13:22:09 2014
@@ -1091,7 +1091,7 @@ public abstract class TopFieldCollector 
    *          the sort criteria (SortFields).
    * @param numHits
    *          the number of results to collect.
-   * @param after
+   * @param after0
    *          only hits after this FieldDoc will be collected
    * @param fillFields
    *          specifies whether the actual field values should be returned on
@@ -1117,7 +1117,7 @@ public abstract class TopFieldCollector 
    *         the sort criteria.
    * @throws IOException if there is a low-level I/O error
    */
-  public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after,
+  public static TopFieldCollector create(Sort sort, int numHits, FieldDoc after0,
       boolean fillFields, boolean trackDocScores, boolean trackMaxScore,
       boolean docsScoredInOrder)
       throws IOException {
@@ -1132,7 +1132,7 @@ public abstract class TopFieldCollector 
 
     FieldValueHitQueue<Entry> queue = FieldValueHitQueue.create(sort.fields, numHits);
 
-    if (after == null) {
+    if (after0 == null) {
       if (queue.getComparators().length == 1) {
         if (docsScoredInOrder) {
           if (trackMaxScore) {
@@ -1172,15 +1172,15 @@ public abstract class TopFieldCollector 
         }
       }
     } else {
-      if (after.fields == null) {
+      if (after0.fields == null) {
         throw new IllegalArgumentException("after.fields wasn't set; you must pass fillFields=true for the previous search");
       }
 
-      if (after.fields.length != sort.getSort().length) {
-        throw new IllegalArgumentException("after.fields has " + after.fields.length + " values but sort has " + sort.getSort().length);
+      if (after0.fields.length != sort.getSort().length) {
+        throw new IllegalArgumentException("after.fields has " + after0.fields.length + " values but sort has " + sort.getSort().length);
       }
 
-      return new PagingFieldCollector(queue, after, numHits, fillFields, trackDocScores, trackMaxScore);
+      return new PagingFieldCollector(queue, after0, numHits, fillFields, trackDocScores, trackMaxScore);
     }
   }
   

Modified: lucene/dev/branches/LUCENE-5716/lucene/ivy-versions.properties
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/ivy-versions.properties?rev=1600647&r1=1600646&r2=1600647&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/ivy-versions.properties (original)
+++ lucene/dev/branches/LUCENE-5716/lucene/ivy-versions.properties Thu Jun  5 13:22:09 2014
@@ -7,6 +7,8 @@
 /cglib/cglib-nodep = 2.2
 /com.adobe.xmp/xmpcore = 5.1.2
 
+/org.aspectj/aspectj = 1.8.0
+
 com.carrotsearch.randomizedtesting.version = 2.1.3
 /com.carrotsearch.randomizedtesting/junit4-ant = ${com.carrotsearch.randomizedtesting.version}
 /com.carrotsearch.randomizedtesting/randomizedtesting-runner = ${com.carrotsearch.randomizedtesting.version}

Modified: lucene/dev/branches/LUCENE-5716/lucene/test-framework/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/test-framework/build.xml?rev=1600647&r1=1600646&r2=1600647&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/test-framework/build.xml (original)
+++ lucene/dev/branches/LUCENE-5716/lucene/test-framework/build.xml Thu Jun  5 13:22:09 2014
@@ -17,7 +17,9 @@
     limitations under the License.
  -->
 
-<project name="test-framework" default="default">
+<project name="test-framework" default="default" 
+	xmlns:ivy="antlib:org.apache.ivy.ant"
+	xmlns:aspectj="antlib:org.aspectj">
   <description>Framework for testing Lucene-based applications</description>
 
   <property name="build.dir" location="../build/test-framework"/>
@@ -35,8 +37,24 @@
   <!-- 
       Specialize compile-core to depend on lucene-core and lucene-codecs compilation.
    -->
-  <target name="compile-core" depends="init,compile-lucene-core,compile-codecs,common.compile-core"
-          description="Compiles test-framework classes"/>
+  <target name="compile-core" depends="init,compile-lucene-core,compile-codecs,common.compile-core,ivy-availability-check,ivy-configure"
+          description="Compiles test-framework classes">
+  	<!-- Compile aspectj aspects. -->
+  	<ivy:cachepath organisation="org.aspectj" module="aspectjtools" revision="1.8.0"
+  	             inline="true" conf="master" type="jar" pathid="aspectjtools.classpath" />
+    <taskdef resource="org/aspectj/antlib.xml" 
+             uri="antlib:org.aspectj" 
+             classpathref="aspectjtools.classpath"/>
+  	
+  	<echo>## AspectJ aspects compilation.</echo>
+  	<aspectj:iajc 
+  		sourceRoots="src/aspects"
+  		outJar="${build.dir}/aspects.jar"
+  		sourceRootCopyFilter="**/*.java,**/*.aj">
+  	  <classpath location="${build.dir}/classes/java" />
+  	  <classpath refid="classpath" />
+  	</aspectj:iajc>
+  </target>
 
   <!-- redefine the clover setup, because we dont want to run clover for the test-framework -->
   <target name="-clover.setup" if="run.clover"/>

Modified: lucene/dev/branches/LUCENE-5716/lucene/test-framework/ivy.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/test-framework/ivy.xml?rev=1600647&r1=1600646&r2=1600647&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/test-framework/ivy.xml (original)
+++ lucene/dev/branches/LUCENE-5716/lucene/test-framework/ivy.xml Thu Jun  5 13:22:09 2014
@@ -34,6 +34,10 @@
       <dependency org="junit" name="junit" rev="${/junit/junit}" transitive="false" conf="default->*;junit4-stdalone->*" />
       <dependency org="com.carrotsearch.randomizedtesting" name="junit4-ant" rev="${/com.carrotsearch.randomizedtesting/junit4-ant}" transitive="false" conf="default->*;junit4-stdalone->*" />
       <dependency org="com.carrotsearch.randomizedtesting" name="randomizedtesting-runner" rev="${/com.carrotsearch.randomizedtesting/randomizedtesting-runner}" transitive="false" conf="default->*;junit4-stdalone->*" />
+      
+      <dependency org="org.aspectj" name="aspectjrt"     rev="${/org.aspectj/aspectj}" transitive="false" />
+      <dependency org="org.aspectj" name="aspectjweaver" rev="${/org.aspectj/aspectj}" transitive="false" />
+      <dependency org="org.aspectj" name="aspectjtools"  rev="${/org.aspectj/aspectj}" transitive="false" />
 
       <exclude org="*" ext="*" matcher="regexp" type="${ivy.exclude.types}"/> 
     </dependencies>

Added: lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/META-INF/aop-ajc.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/META-INF/aop-ajc.xml?rev=1600647&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/META-INF/aop-ajc.xml (added)
+++ lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/META-INF/aop-ajc.xml Thu Jun  5 13:22:09 2014
@@ -0,0 +1,17 @@
+<aspectj>
+  <aspects>
+    <aspect name="org.apache.lucene.util.NoDescriptorLeaksAspect" />
+  </aspects>
+
+  <weaver options="-XnoInline -Xlint:ignore" />
+    
+    <!--
+    <weaver
+    options="
+    -XnoInline
+    -verbose
+    -showWeaveInfo
+    -Xlint:ignore" />
+     -->
+</aspectj>
+

Added: lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/apache/lucene/util/NoDescriptorLeaksAspect.aj
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/apache/lucene/util/NoDescriptorLeaksAspect.aj?rev=1600647&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/apache/lucene/util/NoDescriptorLeaksAspect.aj (added)
+++ lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/aspects/org/apache/lucene/util/NoDescriptorLeaksAspect.aj Thu Jun  5 13:22:09 2014
@@ -0,0 +1,30 @@
+package org.apache.lucene.util;
+
+/*
+ * 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.
+ */
+
+public aspect NoDescriptorLeaksAspect {
+  void around(): execution(void org.apache.lucene.util.TestRuleAspectJAspects.pointcutBeforeSuite(..)) {
+    System.out.println("## Before rule!");
+    proceed();
+  }
+  
+  void around(): execution(void org.apache.lucene.util.TestRuleAspectJAspects.pointcutAfterSuite(..)) {
+    System.out.println("## After rule!");
+    proceed();
+  }  
+}

Modified: lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/AbstractBeforeAfterRule.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/AbstractBeforeAfterRule.java?rev=1600647&r1=1600646&r2=1600647&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/AbstractBeforeAfterRule.java (original)
+++ lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/AbstractBeforeAfterRule.java Thu Jun  5 13:22:09 2014
@@ -28,7 +28,7 @@ import org.junit.runners.model.Statement
  */
 
 /**
- * A {@link TestRule} that guarantees the execution of {@link #after} even
+ * A {@link TestRule} that guarantees the execution of <code>after</code> even
  * if an exception has been thrown from delegate {@link Statement}. This is much
  * like {@link AfterClass} or {@link After} annotations but can be used with
  * {@link RuleChain} to guarantee the order of execution.

Modified: lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java?rev=1600647&r1=1600646&r2=1600647&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java Thu Jun  5 13:22:09 2014
@@ -576,6 +576,7 @@ public abstract class LuceneTestCase ext
     .around(ignoreAfterMaxFailures)
     .around(suiteFailureMarker = new TestRuleMarkFailure())
     .around(new TestRuleAssertionsRequired())
+    .around(new TestRuleAspectJAspects(suiteFailureMarker))
     .around(new TestRuleLimitSysouts(suiteFailureMarker))
     .around(tempFilesCleanupRule = new TestRuleTemporaryFilesCleanup(suiteFailureMarker))
     .around(new StaticFieldsInvariantRule(STATIC_LEAK_THRESHOLD, true) {

Added: lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAspectJAspects.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAspectJAspects.java?rev=1600647&view=auto
==============================================================================
--- lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAspectJAspects.java (added)
+++ lucene/dev/branches/LUCENE-5716/lucene/test-framework/src/java/org/apache/lucene/util/TestRuleAspectJAspects.java Thu Jun  5 13:22:09 2014
@@ -0,0 +1,45 @@
+package org.apache.lucene.util;
+
+import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter;
+
+/*
+ * 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.
+ */
+
+public class TestRuleAspectJAspects extends TestRuleAdapter {
+  private TestRuleMarkFailure failureMarker;
+
+  public TestRuleAspectJAspects(TestRuleMarkFailure failureMarker) {
+    this.failureMarker = failureMarker;
+  }
+  
+  @Override
+  protected void before() throws Throwable {
+    pointcutBeforeSuite();
+  }
+  
+  @Override
+  protected void afterIfSuccessful() throws Throwable {
+    if (failureMarker.wasSuccessful()) {
+      pointcutAfterSuite();
+    }
+  }
+
+  // AspectJ-injected aspects go here.
+
+  private void pointcutBeforeSuite() {}
+  private void pointcutAfterSuite() {}
+}
\ No newline at end of file