This is an automated email from the ASF dual-hosted git repository. zhouxj pushed a commit to branch feature/GEODE-3244 in repository https://gitbox.apache.org/repos/asf/geode.git
commit 90d3a4bf0fdf0227f21857d294352a79911e6d75 Author: zhouxh <[email protected]> AuthorDate: Thu Oct 5 15:01:59 2017 -0700 GEODE-3244: provide a build in LuceneSerializer that flattens objects --- .../geode/cache/lucene/LuceneSerializer.java | 1 + .../repository/serializer/FieldsWithPrefix.java} | 38 +-- .../serializer/FlatFormatSerializer.java | 255 +++++++++++++++++++++ .../serializer/ReflectionLuceneSerializer.java | 10 +- .../NestedObjectSeralizerIntegrationTest.java | 208 +++++++++++++++++ .../serializer/FlatFormatSerializerJUnitTest.java | 41 ++++ .../apache/geode/cache/lucene/test/Customer.java} | 35 ++- .../org/apache/geode/cache/lucene/test/Page.java} | 41 ++-- .../apache/geode/cache/lucene/test/Person.java} | 43 ++-- 9 files changed, 591 insertions(+), 81 deletions(-) diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java index 363867d..ceebc65 100644 --- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java +++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java @@ -32,6 +32,7 @@ public interface LuceneSerializer extends Declarable { * Add the fields of the given value to a set of documents * * @param index lucene index + * @param value user object to be serialized into index */ Collection<Document> toDocuments(LuceneIndex index, Object value); } diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FieldsWithPrefix.java similarity index 56% copy from geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java copy to geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FieldsWithPrefix.java index 363867d..ccab403 100644 --- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java +++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FieldsWithPrefix.java @@ -4,34 +4,34 @@ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance with the License. You may obtain a * copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express * or implied. See the License for the specific language governing permissions and limitations under * the License. */ +package org.apache.geode.cache.lucene.internal.repository.serializer; -package org.apache.geode.cache.lucene; +import java.util.HashSet; -import java.util.Collection; +import org.apache.geode.cache.lucene.LuceneSerializer; -import org.apache.geode.cache.Declarable; -import org.apache.lucene.document.Document; +public class FieldsWithPrefix { + LuceneSerializer serializer; + HashSet<String> topLevelFieldSet; + String prefix; -import org.apache.geode.annotations.Experimental; + FieldsWithPrefix(HashSet<String> topLevelFieldSet, String prefix) { + // TODO initialize serializer + this.topLevelFieldSet = topLevelFieldSet; + this.prefix = prefix; + } -/** - * An interface for writing the fields of an object into a lucene document - */ -@Experimental -public interface LuceneSerializer extends Declarable { - - /** - * Add the fields of the given value to a set of documents - * - * @param index lucene index - */ - Collection<Document> toDocuments(LuceneIndex index, Object value); + @Override + public String toString() { + return "prefix is " + prefix + ", topLevelFieldSet is " + topLevelFieldSet + ":serializer is " + + serializer; + } } diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializer.java new file mode 100644 index 0000000..3f22b3f --- /dev/null +++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializer.java @@ -0,0 +1,255 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.cache.lucene.internal.repository.serializer; + +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; + +import org.apache.geode.cache.lucene.LuceneIndex; +import org.apache.geode.cache.lucene.LuceneSerializer; +import org.apache.geode.cache.lucene.LuceneService; +import org.apache.geode.internal.logging.LogService; +import org.apache.geode.internal.util.concurrent.CopyOnWriteWeakHashMap; +import org.apache.geode.pdx.PdxInstance; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field.Store; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexableField; + +public class FlatFormatSerializer implements LuceneSerializer { + + /** + * keep class to defined fields mapping for each class For example, Customer object based on + * indexed fields name, contact.name, contact.address, contact.homepage.title will become + * <Customer.class, (<name, contact>, prefix=null)>; <Person.class, (<name, address, + * homepage.title>, prefix=contact)>, <Page.class, (<title>, prefix=homepage)> + */ + public HashMap<Class<?>, FieldsWithPrefix> classToFieldsMapping = new HashMap<>(); + + private static final Logger logger = LogService.getLogger(); + + public FlatFormatSerializer() {} + + @Override + public Collection<Document> toDocuments(LuceneIndex index, Object value) { + if (value == null) { + return Collections.emptyList(); + } + + LuceneSerializer mapper = getFieldMapper(value, index.getFieldNames()); + + Collection<Document> docs = mapper.toDocuments(index, value); + if (logger.isDebugEnabled()) { + logger.debug("FlatFormatSerializer.toDocuments:" + docs); + } + + return docs; + } + + /** + * Get the field mapper based on the type of the given object. + */ + public LuceneSerializer getFieldMapper(Object value, String[] indexedFields) { + if (value instanceof PdxInstance) { + throw new IllegalStateException("FlatFormatSerializer does not support PDX"); + } else { + return createClassToFieldsMapping(value, indexedFields, null); + } + } + + /* + * create value's class to its indexed fields mapping + */ + private synchronized LuceneSerializer createClassToFieldsMapping(Object value, + String[] indexedFields, String prefix) { + Class<?> clazz = value.getClass(); + FieldsWithPrefix fieldsWithPrefix = classToFieldsMapping.get(clazz); + if (fieldsWithPrefix == null) { + if (SerializerUtil.supportedPrimitiveTypes().contains(clazz) + && Arrays.asList(indexedFields).contains(LuceneService.REGION_VALUE_FIELD)) { + final PrimitiveSerializer primitiveSerializer = new PrimitiveSerializer(); + final HashSet regionValueFields = new HashSet(); + regionValueFields.add(LuceneService.REGION_VALUE_FIELD); + final FieldsWithPrefix regionValueFieldsWithPrefix = + new FieldsWithPrefix(regionValueFields, null); + regionValueFieldsWithPrefix.serializer = primitiveSerializer; + SerializerUtil.supportedPrimitiveTypes().stream() + .forEach(type -> classToFieldsMapping.put(type, regionValueFieldsWithPrefix)); + return primitiveSerializer; + } + + HashSet<String> topLevelFieldSet = new HashSet<String>(); + fieldsWithPrefix = new FieldsWithPrefix(topLevelFieldSet, prefix); + classToFieldsMapping.put(clazz, fieldsWithPrefix); + } else { + return fieldsWithPrefix.serializer; + } + if (logger.isDebugEnabled()) { + logger.debug("createClassToFieldsMapping:found an undefined class, object value is " + value + + ":indexedFields are " + Arrays.toString(indexedFields) + ", prefix is " + prefix); + } + + // parent field (e.g. contact) to child fields is a 1:N mapping, + // such as contact to <name, address, homepage.title> + HashMap<String, HashSet<String>> parentToChildFieldsMapping = new HashMap(); + for (String f : indexedFields) { + int firstDot = f.indexOf("."); + if (firstDot != -1) { + // found a parent-child field + String parentFieldName = f.substring(0, firstDot); + String childFieldName = f.substring(firstDot + 1); + HashSet<String> childFieldSet = parentToChildFieldsMapping.get(parentFieldName); + if (childFieldSet == null) { + childFieldSet = new HashSet<String>(); + parentToChildFieldsMapping.put(parentFieldName, childFieldSet); + } + childFieldSet.add(childFieldName); + + fieldsWithPrefix.topLevelFieldSet.add(parentFieldName); + } else { + // only found a top level field + fieldsWithPrefix.topLevelFieldSet.add(f); + } + } + if (logger.isDebugEnabled()) { + logger.info("createClassToFieldsMapping:topLevelFieldSet is " + + fieldsWithPrefix.topLevelFieldSet + ":prefix is " + prefix + + ":parentToChildFieldsMap is " + parentToChildFieldsMapping); + } + + String[] fields = fieldsWithPrefix.topLevelFieldSet + .toArray(new String[fieldsWithPrefix.topLevelFieldSet.size()]); + fieldsWithPrefix.serializer = new FlatFormatReflectionLuceneSerializer(clazz, fields, prefix); + + // Now we've got a topLevelFieldSet, such as <name, contact> and a parentToChildFieldsMapping + // such as <contact, <name, address, homepage.title>> + // for each parent field, i.e. contact, recursively process with its childFieldSet, i.e. <name, + // address, homepage.title> + for (Map.Entry<String, HashSet<String>> entry : parentToChildFieldsMapping.entrySet()) { + String parentFieldName = entry.getKey(); + HashSet<String> childFieldSet = entry.getValue(); + String[] childFields = childFieldSet.toArray(new String[childFieldSet.size()]); + + // parentFieldName is to get childValue + Object childValue = getFieldValue(value, parentFieldName); + + // now we've got a child object, find class-fields mapping for its fields + // this time, the parameters are (personObject, <name, address, homepage.title>) + createClassToFieldsMapping(childValue, childFields, parentFieldName); + } + + return fieldsWithPrefix.serializer; + } + + private Object getFieldValue(Object parentValue, String parentFieldName) { + Class<?> parentClazz = parentValue.getClass(); + try { + Field field = parentClazz.getDeclaredField(parentFieldName); + field.setAccessible(true); + return field.get(parentValue); + } catch (Exception e) { + return null; + } + } + + class FlatFormatReflectionLuceneSerializer extends ReflectionLuceneSerializer { + + String prefix; + + public FlatFormatReflectionLuceneSerializer(Class<? extends Object> clazz, + String[] indexedFields, String prefix) { + super(clazz, indexedFields); + this.prefix = prefix; + } + + @Override + public Collection<Document> toDocuments(LuceneIndex index, Object value) { + Document parentDoc = new Document(); + + for (Field field : fields) { + try { + field.setAccessible(true); + Object fieldValue = field.get(value); + if (fieldValue == null) { + continue; + } + Class<?> fieldType = field.getType(); + + if (classToFieldsMapping.containsKey(fieldType)) { + // for each object field value with class defined in classToFieldsMapping, serialize it + // to a document + // then add the document's indexableFields into parentDoc + FieldsWithPrefix fieldsWithPrefix = classToFieldsMapping.get(fieldType); + HashSet<String> topLevelFieldSet = fieldsWithPrefix.topLevelFieldSet; + String[] topLevelFieldNames = + topLevelFieldSet.toArray(new String[topLevelFieldSet.size()]); + + LuceneSerializer childSerializer = getFieldMapper(fieldValue, topLevelFieldNames); + Collection<Document> childDocs = childSerializer.toDocuments(index, fieldValue); + for (Document childDoc : childDocs) { + for (IndexableField oneDocField : childDoc.getFields()) { + parentDoc.add(addPrefixToDocumentField(oneDocField, prefix)); + } + } + } else { + SerializerUtil.addField(parentDoc, + (prefix == null ? field.getName() : prefix + "." + field.getName()), fieldValue); + } + } catch (IllegalArgumentException | IllegalAccessException e) { + if (logger.isDebugEnabled()) { + logger.debug("FlatFormatReflectionLuceneSerializer.toDocuments encountered exception", + e); + } + } + } + if (logger.isDebugEnabled()) { + logger.debug("FlatFormatReflectionLuceneSerializer.toDocuments:" + parentDoc); + } + return Collections.singleton(parentDoc); + } + + private IndexableField addPrefixToDocumentField(IndexableField oneDocField, String prefix) { + if (prefix == null) { + return oneDocField; + } + String newName = prefix + "." + oneDocField.name(); + // do it for each field type used in SerializerUtil.addField() + if (oneDocField instanceof TextField) { + TextField newTextField = + new TextField(newName, ((TextField) oneDocField).stringValue(), Store.NO); + return newTextField; + } + // TODO: we have to change field name for each field type. But LongPoint seems not + // straightforward. + return null; + } + + protected boolean isSupported(Class<?> type) { + return true; + } + + @Override + public String toString() { + return "FlatFormatReflectionLuceneSerializer:prefix is " + prefix + ", fields are " + + Arrays.toString(fields); + } + } +} diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java index 5b9298b..176d4e1 100644 --- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java +++ b/geode-lucene/src/main/java/org/apache/geode/cache/lucene/internal/repository/serializer/ReflectionLuceneSerializer.java @@ -35,9 +35,9 @@ import org.apache.geode.internal.logging.LogService; */ class ReflectionLuceneSerializer implements LuceneSerializer { - private Field[] fields; + protected Field[] fields; - private static final Logger logger = LogService.getLogger(); + protected static final Logger logger = LogService.getLogger(); public ReflectionLuceneSerializer(Class<? extends Object> clazz, String[] indexedFields) { Set<String> fieldSet = new HashSet<String>(); @@ -50,7 +50,7 @@ class ReflectionLuceneSerializer implements LuceneSerializer { while (clazz != Object.class) { for (Field field : clazz.getDeclaredFields()) { Class<?> type = field.getType(); - if (fieldSet.contains(field.getName()) && SerializerUtil.isSupported(type)) { + if (fieldSet.contains(field.getName()) && isSupported(type)) { field.setAccessible(true); foundFields.add(field); } @@ -62,6 +62,10 @@ class ReflectionLuceneSerializer implements LuceneSerializer { this.fields = foundFields.toArray(new Field[foundFields.size()]); } + protected boolean isSupported(Class<?> type) { + return SerializerUtil.isSupported(type); + } + @Override public Collection<Document> toDocuments(LuceneIndex index, Object value) { Document doc = new Document(); diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/NestedObjectSeralizerIntegrationTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/NestedObjectSeralizerIntegrationTest.java new file mode 100644 index 0000000..61abba4 --- /dev/null +++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/NestedObjectSeralizerIntegrationTest.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.cache.lucene; + +import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.INDEX_NAME; +import static org.apache.geode.cache.lucene.test.LuceneTestUtilities.REGION_NAME; +import static org.junit.Assert.assertEquals; + +import java.util.concurrent.TimeUnit; + +import org.apache.logging.log4j.Logger; +import org.apache.geode.cache.Region; +import org.apache.geode.cache.RegionShortcut; +import org.apache.geode.cache.lucene.internal.repository.serializer.FlatFormatSerializer; +import org.apache.geode.cache.lucene.test.Customer; +import org.apache.geode.internal.logging.LogService; +import org.apache.geode.test.junit.categories.IntegrationTest; +import org.apache.lucene.analysis.core.KeywordAnalyzer; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(IntegrationTest.class) +public class NestedObjectSeralizerIntegrationTest extends LuceneIntegrationTest { + + private static int WAIT_FOR_FLUSH_TIME = 10000; + private static final Logger logger = LogService.getLogger(); + + @Test + public void queryOnNestedObject() throws InterruptedException, LuceneQueryException { + // define index on nested objects + luceneService.createIndexFactory().setLuceneSerializer(new FlatFormatSerializer()) + .addField("name").addField("contact.name").addField("contact.email", new KeywordAnalyzer()) + .addField("contact.address").addField("contact.homepage.content") + .addField(LuceneService.REGION_VALUE_FIELD).create(INDEX_NAME, REGION_NAME); + + Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION); + + // insert a few records + region.put("object-13", new Customer("Tommy Jaskcon")); + region.put("object-14", new Customer("Johnny Jackson")); + region.put("object-15", new Customer("Johnny Jackson2")); + region.put("object-16", new Customer("Johnny Jackson21")); + region.put("key-1", "region value 1"); + region.put("key-2", "region value 2"); + region.put("key-3", "region value 3"); + region.put("key-4", "region value 4"); + + LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME); + luceneService.waitUntilFlushed(INDEX_NAME, REGION_NAME, WAIT_FOR_FLUSH_TIME, + TimeUnit.MILLISECONDS); + + // query-1: contact.name + LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "contact.name:jackson2*", "name"); + PageableLuceneQueryResults<Integer, Customer> results = query.findPages(); + assertEquals(2, results.size()); + printResults(results); + + // query-2: name + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "\"Johnni Jackson\"", "contact.name"); + results = query.findPages(); + assertEquals(1, results.size()); + printResults(results); + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "\"Johnni Jackson\"", "name"); + results = query.findPages(); + printResults(results); + assertEquals(0, results.size()); + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "\"Johnny Jackson\"", "name"); + results = query.findPages(); + assertEquals(1, results.size()); + printResults(results); + + // query-3: contact.email with KeywordAnalyzer + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "[email protected]", "contact.email"); + results = query.findPages(); + assertEquals(1, results.size()); + printResults(results); + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "[email protected]", "email"); + results = query.findPages(); + printResults(results); + assertEquals(0, results.size()); + + // query-4: contact.address with standard analyzer + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006", + "contact.address"); + results = query.findPages(); + assertEquals(4, results.size()); + printResults(results); + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006", + "address"); + results = query.findPages(); + printResults(results); + assertEquals(0, results.size()); + + // query-4: contact.homepage.title + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "contact.homepage.content:Hello", "name"); + results = query.findPages(); + printResults(results); + assertEquals(4, results.size()); + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "Hello", + "content"); + results = query.findPages(); + printResults(results); + assertEquals(0, results.size()); + + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "region", + LuceneService.REGION_VALUE_FIELD); + results = query.findPages(); + printResults(results); + assertEquals(4, results.size()); + } + + @Test + public void indexOnNonExistFields() throws InterruptedException, LuceneQueryException { + // define index on nested objects + luceneService.createIndexFactory().setLuceneSerializer(new FlatFormatSerializer()) + .addField("name").addField("contact").addField("contact.page").addField("contact.missing") + .addField("missing2").create(INDEX_NAME, REGION_NAME); + + Region region = createRegion(REGION_NAME, RegionShortcut.PARTITION); + + // insert a few records + region.put("object-13", new Customer("Tommy Jaskcon")); + region.put("object-14", new Customer("Johnny Jackson")); + region.put("object-15", new Customer("Johnny Jackson2")); + region.put("object-16", new Customer("Johnny Jackson21")); + + LuceneIndex index = luceneService.getIndex(INDEX_NAME, REGION_NAME); + luceneService.waitUntilFlushed(INDEX_NAME, REGION_NAME, WAIT_FOR_FLUSH_TIME, + TimeUnit.MILLISECONDS); + + // query-1: contact.name + LuceneQuery query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "Jackson2*", "name"); + PageableLuceneQueryResults<Integer, Customer> results = query.findPages(); + assertEquals(2, results.size()); + printResults(results); + + // query-2: name + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "\"Johnni Jackson\"", "contact.name"); + results = query.findPages(); + assertEquals(0, results.size()); + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "\"Johnny Jackson\"", "name"); + results = query.findPages(); + assertEquals(1, results.size()); + printResults(results); + + // query-3: contact.email with KeywordAnalyzer + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "[email protected]", "contact.email"); + results = query.findPages(); + assertEquals(0, results.size()); + + // query-4: contact.address with standard analyzer + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "97006", + "contact.address"); + results = query.findPages(); + assertEquals(0, results.size()); + + // query-4: contact.homepage.title + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, + "contact.homepage.content:Hello", "name"); + results = query.findPages(); + printResults(results); + assertEquals(0, results.size()); + + // query-5: missing fields + query = luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "*", + "contact.missing"); + results = query.findPages(); + assertEquals(0, results.size()); + query = + luceneService.createLuceneQueryFactory().create(INDEX_NAME, REGION_NAME, "*", "missing2"); + results = query.findPages(); + assertEquals(0, results.size()); + } + + private void printResults(PageableLuceneQueryResults<Integer, Customer> results) { + if (results.size() > 0) { + while (results.hasNext()) { + results.next().stream().forEach(struct -> { + logger.info("Result is:" + struct.getValue()); + }); + } + } + } + +} diff --git a/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializerJUnitTest.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializerJUnitTest.java new file mode 100644 index 0000000..acec39f --- /dev/null +++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/internal/repository/serializer/FlatFormatSerializerJUnitTest.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license + * agreements. See the NOTICE file distributed with this work for additional information regarding + * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. You may obtain a + * copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License + * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express + * or implied. See the License for the specific language governing permissions and limitations under + * the License. + */ +package org.apache.geode.cache.lucene.internal.repository.serializer; + +import org.apache.geode.test.junit.categories.UnitTest; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(UnitTest.class) +public class FlatFormatSerializerJUnitTest { + + @Test + public void shouldParseMultiLayerNestObject() { + String[] fields = new String[] {"name", "contact.name", "contact.email", "contact.revenue", + "contact.address,", "contact.homepage.id", "contact.homepage.title", + "contact.homepage.content"}; + + FlatFormatSerializer serializer = new FlatFormatSerializer(); + } + + @Test + public void shouldNotParseNestedObjectWithoutFields() { + String[] fields = + new String[] {"name", "contact", "contact.homepage", "contact.missing", "missing2"}; + + FlatFormatSerializer serializer = new FlatFormatSerializer(); + } + +} diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Customer.java similarity index 59% copy from geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java copy to geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Customer.java index 363867d..440e0d7 100644 --- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java +++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Customer.java @@ -4,34 +4,31 @@ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance with the License. You may obtain a * copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express * or implied. See the License for the specific language governing permissions and limitations under * the License. */ +package org.apache.geode.cache.lucene.test; -package org.apache.geode.cache.lucene; +import java.io.Serializable; -import java.util.Collection; +public class Customer implements Serializable { + private String name; + private Person contact; // search nested object -import org.apache.geode.cache.Declarable; -import org.apache.lucene.document.Document; + public Customer(String name) { + this.name = name; -import org.apache.geode.annotations.Experimental; + // purposely use Tommi instead of Tommy, Johnni instead of Johnny in contact.name + this.contact = new Person(name.replace('y', 'i')); + } -/** - * An interface for writing the fields of an object into a lucene document - */ -@Experimental -public interface LuceneSerializer extends Declarable { - - /** - * Add the fields of the given value to a set of documents - * - * @param index lucene index - */ - Collection<Document> toDocuments(LuceneIndex index, Object value); + @Override + public String toString() { + return "Customer[name=" + name + ",contact=" + contact + "]"; + } } diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Page.java similarity index 59% copy from geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java copy to geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Page.java index 363867d..bd0dc57 100644 --- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java +++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Page.java @@ -4,34 +4,35 @@ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance with the License. You may obtain a * copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express * or implied. See the License for the specific language governing permissions and limitations under * the License. */ +package org.apache.geode.cache.lucene.test; -package org.apache.geode.cache.lucene; +import java.io.Serializable; -import java.util.Collection; +public class Page implements Serializable { + private int id; + private String title; + private String content; -import org.apache.geode.cache.Declarable; -import org.apache.lucene.document.Document; + public Page(int id) { + this.id = id; + if (id % 2 == 0) { + this.title = "manager"; + } else { + this.title = "developer"; + } + this.content = "Hello world no " + id; + } -import org.apache.geode.annotations.Experimental; - -/** - * An interface for writing the fields of an object into a lucene document - */ -@Experimental -public interface LuceneSerializer extends Declarable { - - /** - * Add the fields of the given value to a set of documents - * - * @param index lucene index - */ - Collection<Document> toDocuments(LuceneIndex index, Object value); + @Override + public String toString() { + return "Page[id=" + id + ",title=" + title + ",content=" + content + "]"; + } } diff --git a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Person.java similarity index 52% copy from geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java copy to geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Person.java index 363867d..dc7b1cf 100644 --- a/geode-lucene/src/main/java/org/apache/geode/cache/lucene/LuceneSerializer.java +++ b/geode-lucene/src/test/java/org/apache/geode/cache/lucene/test/Person.java @@ -4,34 +4,37 @@ * copyright ownership. The ASF licenses this file to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance with the License. You may obtain a * copy of the License at - * + * * http://www.apache.org/licenses/LICENSE-2.0 - * + * * Unless required by applicable law or agreed to in writing, software distributed under the License * is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express * or implied. See the License for the specific language governing permissions and limitations under * the License. */ +package org.apache.geode.cache.lucene.test; -package org.apache.geode.cache.lucene; +import java.io.Serializable; -import java.util.Collection; +public class Person implements Serializable { + private String name; + private String email; + private int revenue; + private String address; + private Page homepage; -import org.apache.geode.cache.Declarable; -import org.apache.lucene.document.Document; + public Person(String name) { + this.name = name; + this.email = name.replace(' ', '.') + "@pivotal.io"; + int length = name.length(); + this.revenue = length * 100; + this.address = "" + length + " NW Greenbrier PKWY, Portland OR 97006"; + this.homepage = new Page(length); + } -import org.apache.geode.annotations.Experimental; - -/** - * An interface for writing the fields of an object into a lucene document - */ -@Experimental -public interface LuceneSerializer extends Declarable { - - /** - * Add the fields of the given value to a set of documents - * - * @param index lucene index - */ - Collection<Document> toDocuments(LuceneIndex index, Object value); + @Override + public String toString() { + return "Person[name=" + name + ",email=" + email + ",revenue=" + revenue + ",address=" + address + + ",homepage=" + homepage + "]"; + } } -- To stop receiving notification emails like this one, please contact "[email protected]" <[email protected]>.
