You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@geode.apache.org by nn...@apache.org on 2016/06/16 23:14:14 UTC

[1/2] incubator-geode git commit: GEODE-11: Added LuceneQueryException to search signature

Repository: incubator-geode
Updated Branches:
  refs/heads/develop 28d2ce069 -> 2fe3a4b2c


GEODE-11: Added LuceneQueryException to search signature

Search will now throw a LuceneQueryException if a query cannot be parsed.

This closes #163


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/2fe3a4b2
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/2fe3a4b2
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/2fe3a4b2

Branch: refs/heads/develop
Commit: 2fe3a4b2cb0399f71b15c99824807d822f5e0ab0
Parents: 786c862
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Thu Jun 16 14:12:42 2016 -0700
Committer: nabarun <nn...@pivotal.io>
Committed: Thu Jun 16 16:14:35 2016 -0700

----------------------------------------------------------------------
 .../gemfire/cache/lucene/LuceneQuery.java       |  2 +-
 .../cache/lucene/LuceneQueryException.java      |  4 +++
 .../cache/lucene/internal/LuceneQueryImpl.java  | 17 +++++++---
 ...IndexCreationPersistenceIntegrationTest.java | 17 +++++++---
 .../lucene/LuceneQueriesIntegrationTest.java    | 34 ++++++--------------
 .../LuceneQueryImplIntegrationTest.java         |  2 +-
 .../distributed/LuceneFunctionJUnitTest.java    |  3 +-
 .../cache/lucene/test/LuceneTestUtilities.java  |  5 +--
 8 files changed, 46 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
index 2de9c0b..93426b9 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQuery.java
@@ -29,7 +29,7 @@ public interface LuceneQuery<K, V> {
   /**
    * Execute the search and get results. 
    */
-  public LuceneQueryResults<K, V> search();
+  public LuceneQueryResults<K, V> search() throws LuceneQueryException;
   
   /**
    * Get page size setting of current query. 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
index 683b799..fb03b4a 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
@@ -33,4 +33,8 @@ public class LuceneQueryException extends GemFireCheckedException {
   public LuceneQueryException(final String message, final Throwable cause) {
     super(message, cause);
   }
+
+  public LuceneQueryException(final Throwable cause) {
+    super(cause);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
index c6b8878..92f1957 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImpl.java
@@ -20,9 +20,11 @@
 package com.gemstone.gemfire.cache.lucene.internal;
 
 import com.gemstone.gemfire.cache.Region;
+import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.FunctionService;
 import com.gemstone.gemfire.cache.execute.ResultCollector;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
@@ -55,7 +57,7 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
   }
 
   @Override
-  public LuceneQueryResults<K, V> search() {
+  public LuceneQueryResults<K, V> search() throws LuceneQueryException {
     TopEntriesCollectorManager manager = new TopEntriesCollectorManager(null, limit);
     LuceneFunctionContext<TopEntriesCollector> context = new LuceneFunctionContext<>(query, indexName, manager, limit);
     TopEntriesFunctionCollector collector = new TopEntriesFunctionCollector(context);
@@ -66,9 +68,16 @@ public class LuceneQueryImpl<K, V> implements LuceneQuery<K, V> {
         .execute(LuceneFunction.ID);
     
     //TODO provide a timeout to the user?
-    TopEntries entries = rc.getResult();
-    
-    return new LuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+    try {
+      TopEntries entries = rc.getResult();
+      return new LuceneQueryResultsImpl<K, V>(entries.getHits(), region, pageSize);
+    } catch(FunctionException e) {
+      if(e.getCause() instanceof LuceneQueryException) {
+        throw new LuceneQueryException(e);
+      } else {
+        throw e;
+      }
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
index 552278e..ca9f4f1 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneIndexCreationPersistenceIntegrationTest.java
@@ -24,9 +24,12 @@ import static junitparams.JUnitParamsRunner.*;
 import static org.junit.Assert.*;
 
 import java.io.File;
+import java.util.concurrent.Callable;
 import java.util.concurrent.TimeUnit;
 import java.util.function.Consumer;
 
+import junit.framework.AssertionFailedError;
+
 import com.jayway.awaitility.Awaitility;
 import junitparams.JUnitParamsRunner;
 import junitparams.Parameters;
@@ -92,7 +95,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
   }
 
   @Test
-  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws ParseException, InterruptedException {
+  public void shouldRecoverPersistentIndexWhenDataStillInQueue() throws Exception {
     createIndex(cache, "field1", "field2");
     Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
@@ -113,7 +116,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
   }
 
   @Test
-  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws ParseException, InterruptedException {
+  public void shouldRecoverPersistentIndexWhenDataIsWrittenToIndex() throws Exception {
     createIndex(cache, "field1", "field2");
     Region dataRegion = cache.createRegionFactory(RegionShortcut.PARTITION_PERSISTENT)
       .create(REGION_NAME);
@@ -132,7 +135,7 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
 
   @Test
   @Parameters(method = "getRegionShortcuts")
-  public void shouldHandleMultipleIndexes(RegionShortcut shortcut) throws ParseException {
+  public void shouldHandleMultipleIndexes(RegionShortcut shortcut) throws Exception {
     LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME+"_1", REGION_NAME, "field1");
     LuceneServiceProvider.get(this.cache).createIndex(INDEX_NAME+"_2", REGION_NAME, "field2");
     Region region = cache.createRegionFactory(shortcut).create(REGION_NAME);
@@ -159,10 +162,14 @@ public class LuceneIndexCreationPersistenceIntegrationTest extends LuceneIntegra
     });
   }
 
-  private void verifyQueryResultSize(String indexName, String regionName, String queryString, String defaultField, int size) throws ParseException {
+  private void verifyQueryResultSize(String indexName, String regionName, String queryString, String defaultField, int size) throws Exception {
     LuceneQuery query = luceneService.createLuceneQueryFactory().create(indexName, regionName, queryString, defaultField);
     Awaitility.await().atMost(60, TimeUnit.SECONDS).until(() -> {
-      assertEquals(size, query.search().size());
+      try {
+        assertEquals(size, query.search().size());
+      } catch(LuceneQueryException e) {
+        throw new RuntimeException(e);
+      }
     });
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index bfb8c88..7aa4cf5 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -17,10 +17,6 @@
 package com.gemstone.gemfire.cache.lucene;
 
 import static com.gemstone.gemfire.cache.lucene.test.LuceneTestUtilities.*;
-import static javax.swing.Action.DEFAULT;
-import static org.hamcrest.Matchers.isA;
-import static org.junit.Assert.assertEquals;
-
 import java.util.HashMap;
 import java.util.Map;
 
@@ -31,7 +27,6 @@ import org.apache.lucene.analysis.core.KeywordAnalyzer;
 import org.apache.lucene.analysis.core.LowerCaseFilter;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.analysis.util.CharTokenizer;
-import org.apache.lucene.queryparser.classic.ParseException;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
@@ -39,9 +34,7 @@ import org.junit.rules.ExpectedException;
 
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.RegionShortcut;
-import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.lucene.test.TestObject;
-import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.pdx.JSONFormatter;
 import com.gemstone.gemfire.pdx.PdxInstance;
 import com.gemstone.gemfire.test.junit.categories.IntegrationTest;
@@ -57,7 +50,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   protected static final String REGION_NAME = "index";
 
   @Test()
-  public void shouldNotTokenizeWordsWithKeywordAnalyzer() throws ParseException {
+  public void shouldNotTokenizeWordsWithKeywordAnalyzer() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     fields.put("field1", new StandardAnalyzer());
     fields.put("field2", new KeywordAnalyzer());
@@ -117,7 +110,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void shouldTokenizeUsingMyCharacterAnalyser() throws ParseException {
+  public void shouldTokenizeUsingMyCharacterAnalyser() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     // not to specify field1's analyzer, it should use standard analyzer
     // Note: fields has to contain "field1", otherwise, field1 will not be tokenized
@@ -145,7 +138,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void shouldAllowNullInFieldValue() throws ParseException {
+  public void shouldAllowNullInFieldValue() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     fields.put("field1", null);
     fields.put("field2", null);
@@ -163,7 +156,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void queryJsonObject() throws ParseException {
+  public void queryJsonObject() throws Exception {
     Map<String, Analyzer> fields = new HashMap<String, Analyzer>();
     fields.put("name", null);
     fields.put("lastName", null);
@@ -186,7 +179,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void shouldAllowQueryOnRegionWithStringValue() throws ParseException {
+  public void shouldAllowQueryOnRegionWithStringValue() throws Exception {
     luceneService.createIndex(INDEX_NAME, REGION_NAME, LuceneService.REGION_VALUE_FIELD);
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
       .create(REGION_NAME);
@@ -199,7 +192,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test()
-  public void throwFunctionExceptionWhenGivenBadQuery() {
+  public void throwFunctionExceptionWhenGivenBadQuery() throws Exception {
     LuceneService luceneService = LuceneServiceProvider.get(cache);
     luceneService.createIndex(INDEX_NAME, REGION_NAME, "text");
     Region region = cache.createRegionFactory(RegionShortcut.PARTITION)
@@ -212,15 +205,8 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
       });
 
 
-    thrown.expect(FunctionException.class);
-    thrown.expectCause(isA(QueryException.class));
-    try {
-      query.search();
-    } catch(FunctionException e) {
-      assertEquals(LuceneQueryException.class, e.getCause().getClass());
-      throw e;
-    }
-
+    thrown.expect(LuceneQueryException.class);
+    query.search();
   }
   
   private PdxInstance insertAJson(Region region, String key) {
@@ -253,14 +239,14 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     return pdx;
   }
 
-  private void verifyQuery(String query, String defaultField, String ... expectedKeys) throws ParseException {
+  private void verifyQuery(String query, String defaultField, String ... expectedKeys) throws Exception {
     final LuceneQuery<String, Object> queryWithStandardAnalyzer = luceneService.createLuceneQueryFactory().create(
       INDEX_NAME, REGION_NAME, query, defaultField);
 
     verifyQueryKeys(queryWithStandardAnalyzer, expectedKeys);
   }
   
-  private void verifyQuery(String query, String DEFAULT_FIELD, HashMap expectedResults) throws ParseException {
+  private void verifyQuery(String query, String DEFAULT_FIELD, HashMap expectedResults) throws Exception {
     final LuceneQuery<String, Object> queryWithStandardAnalyzer = luceneService.createLuceneQueryFactory().create(
       INDEX_NAME, REGION_NAME, query, DEFAULT_FIELD);
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
index c2e9b0c..f748b79 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/LuceneQueryImplIntegrationTest.java
@@ -61,7 +61,7 @@ public class LuceneQueryImplIntegrationTest extends LuceneIntegrationTest {
   }
 
   @Test
-  public void shouldInvokeLuceneFunctionWithCorrectArguments() {
+  public void shouldInvokeLuceneFunctionWithCorrectArguments() throws Exception {
     // Register a fake function to observe the function invocation
     FunctionService.unregisterFunction(LuceneFunction.ID);
     TestLuceneFunction function = new TestLuceneFunction();

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
index 178fae2..26c4d62 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunctionJUnitTest.java
@@ -31,6 +31,7 @@ import java.util.List;
 import com.gemstone.gemfire.cache.Region;
 import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryFactory;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.internal.InternalLuceneService;
@@ -240,7 +241,7 @@ public class LuceneFunctionJUnitTest {
     when(mockContext.getDataSet()).thenReturn(mockRegion);
     when(mockContext.getArguments()).thenReturn(searchArgs);
     when(mockContext.<TopEntriesCollector>getResultSender()).thenReturn(mockResultSender);
-    when(queryProvider.getQuery(eq(mockIndex))).thenThrow(QueryException.class);
+    when(queryProvider.getQuery(eq(mockIndex))).thenThrow(LuceneQueryException.class);
     LuceneFunction function = new LuceneFunction();
 
     function.execute(mockContext);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/2fe3a4b2/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
index d7150d9..c1f540d 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/test/LuceneTestUtilities.java
@@ -33,6 +33,7 @@ import com.gemstone.gemfire.cache.asyncqueue.AsyncEventQueue;
 import com.gemstone.gemfire.cache.asyncqueue.internal.AsyncEventQueueImpl;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
 import com.gemstone.gemfire.cache.lucene.LuceneQuery;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryResults;
 import com.gemstone.gemfire.cache.lucene.LuceneResultStruct;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
@@ -90,7 +91,7 @@ public class LuceneTestUtilities {
   /**
    * Verify that a query returns the expected list of keys. Ordering is ignored.
    */
-  public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) {
+  public static <K> void  verifyQueryKeys(LuceneQuery<K,Object> query,K ... expectedKeys) throws LuceneQueryException {
     Set<K> expectedKeySet = new HashSet<>(Arrays.asList(expectedKeys));
     Set<K> actualKeySet = new HashSet<>();
     final LuceneQueryResults<K, Object> results = query.search();
@@ -104,7 +105,7 @@ public class LuceneTestUtilities {
   /**
    * Verify that a query returns the expected map of key-value. Ordering is ignored.
    */
-  public static <K> void verifyQueryKeyAndValues(LuceneQuery<K,Object> query, HashMap expectedResults) {
+  public static <K> void verifyQueryKeyAndValues(LuceneQuery<K,Object> query, HashMap expectedResults) throws LuceneQueryException {
     HashMap actualResults = new HashMap<>();
     final LuceneQueryResults<K, Object> results = query.search();
     while(results.hasNextPage()) {


[2/2] incubator-geode git commit: GEODE-11: Removed ParseException from LuceneQueryFactory.create, added LuceneQueryException

Posted by nn...@apache.org.
GEODE-11: Removed ParseException from LuceneQueryFactory.create, added LuceneQueryException

Adding a new exception to wrap lucene exceptions called LuceneQueryException. Removed lucene exceptions from the public API.


Project: http://git-wip-us.apache.org/repos/asf/incubator-geode/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-geode/commit/786c862e
Tree: http://git-wip-us.apache.org/repos/asf/incubator-geode/tree/786c862e
Diff: http://git-wip-us.apache.org/repos/asf/incubator-geode/diff/786c862e

Branch: refs/heads/develop
Commit: 786c862e1c158859b90d52ee0d053507b65e8a01
Parents: 28d2ce0
Author: Aparna Dharmakkan <ad...@pivotal.io>
Authored: Thu Jun 16 11:20:13 2016 -0700
Committer: nabarun <nn...@pivotal.io>
Committed: Thu Jun 16 16:14:35 2016 -0700

----------------------------------------------------------------------
 .../cache/lucene/LuceneQueryException.java      | 36 ++++++++++++++++++++
 .../cache/lucene/LuceneQueryFactory.java        |  4 +--
 .../cache/lucene/LuceneQueryProvider.java       |  5 +--
 .../lucene/internal/StringQueryProvider.java    |  8 +++--
 .../internal/distributed/LuceneFunction.java    |  3 +-
 .../lucene/LuceneQueriesIntegrationTest.java    |  4 +--
 .../internal/StringQueryProviderJUnitTest.java  | 13 +++----
 .../IndexRepositoryImplPerformanceTest.java     |  5 +--
 8 files changed, 59 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
new file mode 100644
index 0000000..683b799
--- /dev/null
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryException.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package com.gemstone.gemfire.cache.lucene;
+
+import com.gemstone.gemfire.GemFireCheckedException;
+
+/**
+ * Thrown when a lucene query fails.
+ */
+public class LuceneQueryException extends GemFireCheckedException {
+
+  public LuceneQueryException(final String message) {
+    super(message);
+  }
+
+  public LuceneQueryException(final String message, final Throwable cause) {
+    super(message, cause);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java
index a7844a2..8e36bbb 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryFactory.java
@@ -80,10 +80,8 @@ public interface LuceneQueryFactory {
    * @param <K> the key type in the query results
    * @param <V> the value type in the query results
    * @return LuceneQuery object
-   * @throws ParseException
    */
-  public <K, V> LuceneQuery<K, V> create(String indexName, String regionName, String queryString, String defaultField)
-      throws ParseException;
+  public <K, V> LuceneQuery<K, V> create(String indexName, String regionName, String queryString, String defaultField);
 
   /**
    * Creates a wrapper object for Lucene's Query object. This {@link LuceneQuery} builder method could be used in

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
index 92a3a1c..7f1c269 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/LuceneQueryProvider.java
@@ -39,8 +39,9 @@ public interface LuceneQueryProvider extends Serializable {
   /**
    * @return A Lucene Query object which could be used for executing Lucene Search on indexed data
    * @param index local lucene index the query is being constructed against.
-   * @throws QueryException if the provider fails to construct the query object
+   * @throws LuceneQueryException if the provider fails to construct the query object
    */
-  public Query getQuery(LuceneIndex index) throws QueryException;
+
+  public Query getQuery(LuceneIndex index) throws LuceneQueryException;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
index 3f121ec..c5d145e 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProvider.java
@@ -29,7 +29,10 @@ import org.apache.lucene.queryparser.flexible.standard.StandardQueryParser;
 import org.apache.lucene.search.Query;
 
 import com.gemstone.gemfire.DataSerializer;
+import com.gemstone.gemfire.GemFireCheckedException;
+import com.gemstone.gemfire.GemFireException;
 import com.gemstone.gemfire.cache.lucene.LuceneIndex;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.internal.DataSerializableFixedID;
@@ -64,10 +67,9 @@ public class StringQueryProvider implements LuceneQueryProvider, DataSerializabl
   }
 
   @Override
-  public synchronized Query getQuery(LuceneIndex index) throws QueryException {
+  public synchronized Query getQuery(LuceneIndex index) throws LuceneQueryException {
     if (luceneQuery == null) {
       String[] fields = index.getFieldNames();
-
       LuceneIndexImpl indexImpl = (LuceneIndexImpl) index;
       StandardQueryParser parser = new StandardQueryParser(indexImpl.getAnalyzer());
       try {
@@ -77,7 +79,7 @@ public class StringQueryProvider implements LuceneQueryProvider, DataSerializabl
         }
       } catch (QueryNodeException e) {
         logger.debug("Query node exception:" + query, e);
-        throw new QueryException(e);
+        throw new LuceneQueryException("Malformed lucene query: " + query, e);
       }
     }
     return luceneQuery;

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java
index 9567305..3c6c0d2 100644
--- a/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java
+++ b/geode-lucene/src/main/java/com/gemstone/gemfire/cache/lucene/internal/distributed/LuceneFunction.java
@@ -32,6 +32,7 @@ import com.gemstone.gemfire.cache.execute.FunctionContext;
 import com.gemstone.gemfire.cache.execute.FunctionException;
 import com.gemstone.gemfire.cache.execute.RegionFunctionContext;
 import com.gemstone.gemfire.cache.execute.ResultSender;
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.cache.lucene.LuceneQueryProvider;
 import com.gemstone.gemfire.cache.lucene.LuceneService;
 import com.gemstone.gemfire.cache.lucene.LuceneServiceProvider;
@@ -80,7 +81,7 @@ public class LuceneFunction extends FunctionAdapter implements InternalEntity {
     Query query = null;
     try {
       query = queryProvider.getQuery(index);
-    } catch (QueryException e) {
+    } catch (LuceneQueryException e) {
       logger.warn("", e);
       throw new FunctionException(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
index 674bc9c..bfb8c88 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/LuceneQueriesIntegrationTest.java
@@ -208,7 +208,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     //Create a query that throws an exception
     final LuceneQuery<Object, Object> query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME,
       (index) -> {
-        throw new QueryException("Bad query");
+        throw new LuceneQueryException("Bad query");
       });
 
 
@@ -217,7 +217,7 @@ public class LuceneQueriesIntegrationTest extends LuceneIntegrationTest {
     try {
       query.search();
     } catch(FunctionException e) {
-      assertEquals(QueryException.class, e.getCause().getClass());
+      assertEquals(LuceneQueryException.class, e.getCause().getClass());
       throw e;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
index a08875a..332ce35 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/StringQueryProviderJUnitTest.java
@@ -32,7 +32,8 @@ import org.junit.experimental.categories.Category;
 import org.mockito.Mockito;
 
 import com.gemstone.gemfire.CopyHelper;
-import com.gemstone.gemfire.cache.query.QueryException;
+
+import com.gemstone.gemfire.cache.lucene.LuceneQueryException;
 import com.gemstone.gemfire.test.junit.categories.UnitTest;
 
 @Category(UnitTest.class)
@@ -52,7 +53,7 @@ public class StringQueryProviderJUnitTest {
   }
 
   @Test
-  public void testQueryConstruction() throws QueryException {
+  public void testQueryConstruction() throws LuceneQueryException {
     StringQueryProvider provider = new StringQueryProvider("foo:bar", DEFAULT_FIELD);
     Query query = provider.getQuery(mockIndex);
     Assert.assertNotNull(query);
@@ -61,15 +62,15 @@ public class StringQueryProviderJUnitTest {
 
   @Test
   @Ignore("Custom analyzer not yet supported, this is a duplicate test right now")
-  public void usesCustomAnalyzer() throws QueryException {
+  public void usesCustomAnalyzer() throws LuceneQueryException {
     StringQueryProvider provider = new StringQueryProvider("findThis", DEFAULT_FIELD);
     Query query = provider.getQuery(mockIndex);
     Assert.assertNotNull(query);
     assertEquals("field-1:findthis field-2:findthis", query.toString());
   }
 
-  @Test(expected = QueryException.class)
-  public void errorsOnMalformedQueryString() throws QueryException {
+  @Test(expected = LuceneQueryException.class)
+  public void errorsOnMalformedQueryString() throws LuceneQueryException {
     StringQueryProvider provider = new StringQueryProvider("invalid:lucene:query:string", DEFAULT_FIELD);
     provider.getQuery(mockIndex);
   }
@@ -83,7 +84,7 @@ public class StringQueryProviderJUnitTest {
   }
 
   @Test
-  public void defaultFieldParameterShouldBeUsedByQuery() throws QueryException {
+  public void defaultFieldParameterShouldBeUsedByQuery() throws LuceneQueryException {
     StringQueryProvider provider = new StringQueryProvider("findThis",  "field-2");
     Query query = provider.getQuery(mockIndex);
     Assert.assertNotNull(query);

http://git-wip-us.apache.org/repos/asf/incubator-geode/blob/786c862e/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
----------------------------------------------------------------------
diff --git a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
index 5e1a104..61f0ec4 100644
--- a/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
+++ b/geode-lucene/src/test/java/com/gemstone/gemfire/cache/lucene/internal/repository/IndexRepositoryImplPerformanceTest.java
@@ -32,7 +32,6 @@ import com.gemstone.gemfire.cache.lucene.internal.filesystem.ChunkKey;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.File;
 import com.gemstone.gemfire.cache.lucene.internal.filesystem.FileSystemStats;
 import com.gemstone.gemfire.cache.lucene.internal.repository.serializer.HeterogeneousLuceneSerializer;
-import com.gemstone.gemfire.cache.query.QueryException;
 import com.gemstone.gemfire.test.junit.categories.PerformanceTest;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
@@ -50,6 +49,7 @@ import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -59,6 +59,7 @@ import java.util.concurrent.TimeUnit;
 
 import static com.gemstone.gemfire.distributed.ConfigurationProperties.*;
 
+
 /**
  * Microbenchmark of the IndexRepository to compare an
  * IndexRepository built on top of cache with a 
@@ -190,7 +191,7 @@ public class IndexRepositoryImplPerformanceTest {
         LuceneQuery<Object, Object> luceneQuery = service.createLuceneQueryFactory().create("index", "/region", new LuceneQueryProvider() {
           
           @Override
-          public Query getQuery(LuceneIndex index) throws QueryException {
+          public Query getQuery(LuceneIndex index) throws LuceneQueryException {
             return query;
           }
         });