Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id AAC76200C8C for ; Tue, 2 May 2017 01:32:56 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id A6C78160BC1; Mon, 1 May 2017 23:32:56 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 6B77B160BDE for ; Tue, 2 May 2017 01:32:53 +0200 (CEST) Received: (qmail 91988 invoked by uid 500); 1 May 2017 23:32:49 -0000 Mailing-List: contact commits-help@geode.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@geode.apache.org Delivered-To: mailing list commits@geode.apache.org Received: (qmail 90712 invoked by uid 99); 1 May 2017 23:32:48 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 01 May 2017 23:32:48 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id C21C9E9663; Mon, 1 May 2017 23:32:48 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: klund@apache.org To: commits@geode.apache.org Date: Mon, 01 May 2017 23:33:15 -0000 Message-Id: <2e410c52d38f464699ffad07557a4920@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [28/37] geode git commit: GEODE-2632: change dependencies on GemFireCacheImpl to InternalCache archived-at: Mon, 01 May 2017 23:32:56 -0000 http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndex.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndex.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndex.java index 023fbb4..58bd2bf 100755 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndex.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/HashIndex.java @@ -12,10 +12,9 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ - package org.apache.geode.cache.query.internal.index; -import it.unimi.dsi.fastutil.objects.Object2ObjectOpenHashMap; +import static org.apache.geode.internal.lang.SystemUtils.*; import java.util.ArrayList; import java.util.Collection; @@ -25,14 +24,14 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; -import org.apache.geode.cache.query.internal.parse.OQLLexerTokenTypes; +import it.unimi.dsi.fastutil.objects.Object2ObjectOpenHashMap; import org.apache.logging.log4j.Logger; -import org.apache.geode.cache.Cache; import org.apache.geode.cache.EntryDestroyedException; import org.apache.geode.cache.Region; import org.apache.geode.cache.RegionAttributes; @@ -63,12 +62,13 @@ import org.apache.geode.cache.query.internal.RuntimeIterator; import org.apache.geode.cache.query.internal.Support; import org.apache.geode.cache.query.internal.index.HashIndex.IMQEvaluator.HashIndexComparator; import org.apache.geode.cache.query.internal.index.IndexStore.IndexStoreEntry; +import org.apache.geode.cache.query.internal.parse.OQLLexerTokenTypes; import org.apache.geode.cache.query.internal.types.StructTypeImpl; import org.apache.geode.cache.query.internal.types.TypeUtils; import org.apache.geode.cache.query.types.ObjectType; -import org.apache.geode.internal.Assert; import org.apache.geode.internal.cache.CachedDeserializable; import org.apache.geode.internal.cache.GemFireCacheImpl; +import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.LocalRegion; import org.apache.geode.internal.cache.RegionEntry; import org.apache.geode.internal.cache.Token; @@ -82,11 +82,10 @@ import org.apache.geode.internal.offheap.StoredObject; * called explicitly with createHashIndex It requires the indexed expression be a path expression * and the from clause has only one iterator. This implies there is only one value in the index for * each region entry. - * + *

* This index does not support the storage of projection attributes. - * + *

* Currently this implementation only supports an index on a region path. - * */ public class HashIndex extends AbstractIndex { private static final Logger logger = LogService.getLogger(); @@ -118,7 +117,7 @@ public class HashIndex extends AbstractIndex { /** * Create a HashIndex that can be used when executing queries. - * + * * @param indexName the name of this index, used for statistics collection * @param indexedExpression the expression to index on, a function dependent on region entries * individually, limited to a path expression. @@ -149,7 +148,7 @@ public class HashIndex extends AbstractIndex { /** * Get the index type - * + * * @return the type of index */ public IndexType getType() { @@ -178,14 +177,10 @@ public class HashIndex extends AbstractIndex { /** * Add/Updates the index forward and reverse map. If index key for a RegionEntry is found same as * previous key no update is performed. - * + * * This also updates the {@link IndexStatistics} numKeys and numValues as and when appropriate. * One thing to notice though is no increment in numValues is performed if old key and new index * key are found equal using {@link Object#equals(Object)}. - * - * @param key - * @param entry - * @throws IMQException */ private void basicAddMapping(Object key, RegionEntry entry) throws IMQException { @@ -288,11 +283,10 @@ public class HashIndex extends AbstractIndex { * inserted for the RegionEntry. In case of update only forward map is cleared of old key and NO * update is performed on reverse map as that has already been done during * {@link HashIndex#basicAddMapping(Object, RegionEntry)}. - * + * * @param key - Index key. * @param entry RegionEntry for which is being updated by user. * @param updateReverseMap true only when RegionEntry is invalidated/destroyed. - * @throws IMQException */ private void basicRemoveMapping(Object key, RegionEntry entry, boolean updateReverseMap) throws IMQException { @@ -359,6 +353,7 @@ public class HashIndex extends AbstractIndex { outerEntry = (Map.Entry) outer.next(); // } outerKey = outerEntry.getKey(); + // TODO: eliminate use of labels inner: while (!incrementInner || inner.hasNext()) { if (incrementInner) { innerEntry = inner.next(); @@ -410,16 +405,8 @@ public class HashIndex extends AbstractIndex { * used. Like, if condition is "p.ID = e.ID", {@link IndexInfo} will contain Left as p.ID, Right * as e.ID and operator as TOK_EQ. This method will evaluate p.ID OR e.ID based on if it is inner * or outer RegionEntry, and verify the p.ID = e.ID. - * - * @param entry - * @param context - * @param indexInfo - * @param keyVal + * * @return true if entry value and index value are consistent. - * @throws FunctionDomainException - * @throws TypeMismatchException - * @throws NameResolutionException - * @throws QueryInvocationTargetException */ private boolean verifyInnerAndOuterEntryValues(RegionEntry entry, ExecutionContext context, IndexInfo indexInfo, Object keyVal) throws FunctionDomainException, TypeMismatchException, @@ -433,57 +420,6 @@ public class HashIndex extends AbstractIndex { return evaluateEntry(indexInfo, context, keyVal); } - // TODO:Decsribe & test the function - /** - * @param outerEntries is a Set - * @param innerEntries is a Set - * @param key - * @throws QueryInvocationTargetException - * @throws NameResolutionException - * @throws TypeMismatchException - * @throws FunctionDomainException - */ - private void populateListForEquiJoin(List list, Collection outerEntries, Collection innerEntries, - ExecutionContext context, Object key) throws FunctionDomainException, TypeMismatchException, - NameResolutionException, QueryInvocationTargetException { - - Assert.assertTrue((outerEntries != null && innerEntries != null), - "OuterEntries or InnerEntries must not be null"); - - Object values[][] = new Object[2][]; - int j = 0; - Iterator itr = null; - while (j < 2) { - if (j == 0) { - itr = outerEntries.iterator(); - } else { - itr = innerEntries.iterator(); - } - // TODO :Asif Identify appropriate size of the List - - // extract the values from the RegionEntries - List dummy = new ArrayList(); - RegionEntry re = null; - while (itr.hasNext()) { - re = (RegionEntry) itr.next(); - // Bug#41010: We need to verify if Inner and Outer Entries - // are consistent with index key values. - boolean ok = true; - if (re.isUpdateInProgress()) { - IndexInfo[] indexInfo = (IndexInfo[]) context.cacheGet(CompiledValue.INDEX_INFO); - IndexInfo indInfo = (j == 0) ? indexInfo[0] : indexInfo[1]; - - ok = verifyInnerAndOuterEntryValues(re, context, indInfo, key); - } - if (ok) { - dummy.add(getTargetObject(re)); - } - } - dummy.toArray(values[j++] = new Object[dummy.size()]); - } - list.add(values); - } - public int getSizeEstimate(Object key, int operator, int matchLevel) throws TypeMismatchException { // Get approx size; @@ -534,21 +470,19 @@ public class HashIndex extends AbstractIndex { int limit = -1; Boolean applyLimit = (Boolean) context.cacheGet(CompiledValue.CAN_APPLY_LIMIT_AT_INDEX); - if (applyLimit != null && applyLimit.booleanValue()) { - limit = ((Integer) context.cacheGet(CompiledValue.RESULT_LIMIT)).intValue(); + if (applyLimit != null && applyLimit) { + limit = (Integer) context.cacheGet(CompiledValue.RESULT_LIMIT); } Boolean orderByClause = (Boolean) context.cacheGet(CompiledValue.CAN_APPLY_ORDER_BY_AT_INDEX); boolean applyOrderBy = false; - boolean asc = true; List orderByAttrs = null; - boolean multiColOrderBy = false; - if (orderByClause != null && orderByClause.booleanValue()) { + if (orderByClause != null && orderByClause) { orderByAttrs = (List) context.cacheGet(CompiledValue.ORDERBY_ATTRIB); CompiledSortCriterion csc = (CompiledSortCriterion) orderByAttrs.get(0); - asc = !csc.getCriterion(); + boolean asc = !csc.getCriterion(); applyOrderBy = true; - multiColOrderBy = orderByAttrs.size() > 1; + boolean multiColOrderBy = orderByAttrs.size() > 1; } evaluate(key, operator, results, iterOps, runtimeItr, context, keysToRemove, projAttrib, intermediateResults, isIntersection, limit, applyOrderBy, orderByAttrs); @@ -623,8 +557,8 @@ public class HashIndex extends AbstractIndex { } @Override - void instantiateEvaluator(IndexCreationHelper ich) { - this.evaluator = new IMQEvaluator(ich); + void instantiateEvaluator(IndexCreationHelper indexCreationHelper) { + this.evaluator = new IMQEvaluator(indexCreationHelper); this.entriesSet.setEvaluator((HashIndex.IMQEvaluator) evaluator); this.comparator = ((IMQEvaluator) evaluator).comparator; } @@ -747,18 +681,11 @@ public class HashIndex extends AbstractIndex { * This evaluates the left and right side of a where condition for which this Index was used. * Like, if condition is "ID > 1", {@link IndexInfo} will contain Left as ID, Right as '1' and * operator as TOK_GT. This method will evaluate ID from region entry value and verify the ID > 1. - * + * * Note: IndexInfo is created for each query separately based on the condition being evaluated * using the Index. - * - * @param indexInfo - * @param context - * @param keyVal + * * @return true if RegionEntry value satisfies the where condition (contained in IndexInfo). - * @throws FunctionDomainException - * @throws TypeMismatchException - * @throws NameResolutionException - * @throws QueryInvocationTargetException */ private boolean evaluateEntry(IndexInfo indexInfo, ExecutionContext context, Object keyVal) throws FunctionDomainException, TypeMismatchException, NameResolutionException, @@ -787,7 +714,7 @@ public class HashIndex extends AbstractIndex { if (left == null && right == null) { return Boolean.TRUE; } else { - return ((Boolean) TypeUtils.compare(left, right, operator)).booleanValue(); + return (Boolean) TypeUtils.compare(left, right, operator); } } @@ -797,7 +724,8 @@ public class HashIndex extends AbstractIndex { */ private Object getTargetObject(RegionEntry entry) { if (this.indexOnValues) { - Object o = entry.getValue((LocalRegion) getRegion()); // OFFHEAP: incrc, deserialize, decrc + // OFFHEAP: incrc, deserialize, decrc + Object o = entry.getValue((LocalRegion) getRegion()); try { if (o == Token.INVALID) { return null; @@ -805,7 +733,7 @@ public class HashIndex extends AbstractIndex { if (o instanceof CachedDeserializable) { return ((CachedDeserializable) o).getDeserializedForReading(); } - } catch (EntryDestroyedException ede) { + } catch (EntryDestroyedException ignored) { return null; } return o; @@ -829,7 +757,7 @@ public class HashIndex extends AbstractIndex { } else if (o instanceof CachedDeserializable) { o = ((CachedDeserializable) o).getDeserializedForReading(); } - } catch (EntryDestroyedException ede) { + } catch (EntryDestroyedException ignored) { return Token.INVALID; } return o; @@ -861,61 +789,33 @@ public class HashIndex extends AbstractIndex { } public String dump() { - StringBuffer sb = new StringBuffer(toString()).append(" {\n"); - // sb.append("Null Values\n"); - // Iterator nI = nullMappedEntries.iterator(); - // while (nI.hasNext()) { - // RegionEntry e = (RegionEntry) nI.next(); - // Object value = getTargetObject(e); - // sb.append(" RegionEntry.key = ").append(e.getKey()); - // sb.append(" Value.type = ").append(value.getClass().getName()); - // if (value instanceof Collection) { - // sb.append(" Value.size = ").append(((Collection) value).size()); - // } - // sb.append("\n"); - // } - // sb.append(" -----------------------------------------------\n"); - // sb.append("Undefined Values\n"); - // Iterator uI = undefinedMappedEntries.iterator(); - // while (uI.hasNext()) { - // RegionEntry e = (RegionEntry) uI.next(); - // Object value = getTargetObject(e); - // sb.append(" RegionEntry.key = ").append(e.getKey()); - // sb.append(" Value.type = ").append(value.getClass().getName()); - // if (value instanceof Collection) { - // sb.append(" Value.size = ").append(((Collection) value).size()); - // } - // sb.append("\n"); - // } - sb.append(" -----------------------------------------------\n"); - Iterator i1 = this.entriesSet.iterator(); - while (i1.hasNext()) { - Map.Entry indexEntry = (Map.Entry) i1.next(); - sb.append(" Key = " + indexEntry.getKey()).append("\n"); - sb.append(" Value Type = ").append(" " + indexEntry.getValue().getClass().getName()) - .append("\n"); + StringBuilder sb = new StringBuilder(toString()).append(" {").append(getLineSeparator()); + sb.append(" -----------------------------------------------").append(getLineSeparator()); + for (Object anEntriesSet : this.entriesSet) { + Entry indexEntry = (Entry) anEntriesSet; + sb.append(" Key = ").append(indexEntry.getKey()).append(getLineSeparator()); + sb.append(" Value Type = ").append(' ').append(indexEntry.getValue().getClass().getName()) + .append(getLineSeparator()); if (indexEntry.getValue() instanceof Collection) { - sb.append(" Value Size = ").append(" " + ((Collection) indexEntry.getValue()).size()) - .append("\n"); + sb.append(" Value Size = ").append(' ').append(((Collection) indexEntry.getValue()).size()) + .append(getLineSeparator()); } else if (indexEntry.getValue() instanceof RegionEntry) { - sb.append(" Value Size = ").append(" " + 1).append("\n"); + sb.append(" Value Size = ").append(" " + 1).append(getLineSeparator()); } else { throw new AssertionError("value instance of " + indexEntry.getValue().getClass().getName()); } Collection entrySet = regionEntryCollection(indexEntry.getValue()); - Iterator i2 = entrySet.iterator(); - while (i2.hasNext()) { - RegionEntry e = (RegionEntry) i2.next(); + for (Object anEntrySet : entrySet) { + RegionEntry e = (RegionEntry) anEntrySet; Object value = getTargetObject(e); sb.append(" RegionEntry.key = ").append(e.getKey()); sb.append(" Value.type = ").append(value.getClass().getName()); if (value instanceof Collection) { sb.append(" Value.size = ").append(((Collection) value).size()); } - sb.append("\n"); - // sb.append(" Value.type = ").append(value).append("\n"); + sb.append(getLineSeparator()); } - sb.append(" -----------------------------------------------\n"); + sb.append(" -----------------------------------------------").append(getLineSeparator()); } sb.append("}// Index ").append(getName()).append(" end"); return sb.toString(); @@ -1038,21 +938,21 @@ public class HashIndex extends AbstractIndex { } public String toString() { - StringBuffer sb = new StringBuffer(); - sb.append("No Keys = ").append(getNumberOfKeys()).append("\n"); - sb.append("No Values = ").append(getNumberOfValues()).append("\n"); - sb.append("No Uses = ").append(getTotalUses()).append("\n"); - sb.append("No Updates = ").append(getNumUpdates()).append("\n"); - sb.append("Total Update time = ").append(getTotalUpdateTime()).append("\n"); + StringBuilder sb = new StringBuilder(); + sb.append("No Keys = ").append(getNumberOfKeys()).append(getLineSeparator()); + sb.append("No Values = ").append(getNumberOfValues()).append(getLineSeparator()); + sb.append("No Uses = ").append(getTotalUses()).append(getLineSeparator()); + sb.append("No Updates = ").append(getNumUpdates()).append(getLineSeparator()); + sb.append("Total Update time = ").append(getTotalUpdateTime()).append(getLineSeparator()); return sb.toString(); } } class IMQEvaluator implements IndexedExpressionEvaluator { - private Cache cache; + private final InternalCache cache; private List fromIterators = null; private CompiledValue indexedExpr = null; - final private String[] canonicalIterNames; + private final String[] canonicalIterNames; private ObjectType indexResultSetType = null; private Region rgn = null; private Map dependencyGraph = null; @@ -1098,7 +998,7 @@ public class HashIndex extends AbstractIndex { this.canonicalIterNames = ((FunctionalIndexCreationHelper) helper).canonicalizedIteratorNames; this.rgn = helper.getRegion(); - // The modified iterators for optmizing Index cxreation + // The modified iterators for optimizing Index creation isFirstItrOnEntry = ((FunctionalIndexCreationHelper) helper).isFirstIteratorRegionEntry; additionalProj = ((FunctionalIndexCreationHelper) helper).additionalProj; Object params1[] = {new QRegion(rgn, false)}; @@ -1173,7 +1073,7 @@ public class HashIndex extends AbstractIndex { /** * This function is used for creating Index data at the start - * + * */ public void initializeIndex(boolean loadEntries) throws IMQException { this.initEntriesUpdated = 0; @@ -1278,7 +1178,6 @@ public class HashIndex extends AbstractIndex { /** * @param add true if adding to index, false if removing - * @param context */ private void doNestedIterations(int level, boolean add, ExecutionContext context) throws TypeMismatchException, AmbiguousNameException, FunctionDomainException, @@ -1302,7 +1201,6 @@ public class HashIndex extends AbstractIndex { /** * @param add true if adding, false if removing from index - * @param context */ private void applyProjection(boolean add, ExecutionContext context) throws FunctionDomainException, TypeMismatchException, NameResolutionException, @@ -1375,33 +1273,30 @@ public class HashIndex extends AbstractIndex { Object params[] = {dQRegion}; ExecutionContext context = new ExecutionContext(params, this.cache); context.newScope(IndexCreationHelper.INDEX_QUERY_SCOPE_ID); - try { - if (this.dependencyGraph != null) { - context.setDependencyGraph(dependencyGraph); - } - for (int i = 0; i < this.iteratorSize; i++) { - CompiledIteratorDef iterDef = (CompiledIteratorDef) fromIterators.get(i); - // We are re-using the same ExecutionContext on every evaluate -- this - // is not how ExecutionContext was intended to be used. - // Asif: Compute the dependency only once. The call to methods of this - // class are thread safe as for update lock on Index is taken . - if (this.dependencyGraph == null) { - iterDef.computeDependencies(context); - } - RuntimeIterator rIter = iterDef.getRuntimeIterator(context); - context.addToIndependentRuntimeItrMapForIndexCreation(iterDef); - context.bindIterator(rIter); - } - // Save the dependency graph for future updates. - if (dependencyGraph == null) { - dependencyGraph = context.getDependencyGraph(); - } - - Support.Assert(this.indexResultSetType != null, - "IMQEvaluator::evaluate:The StrcutType should have been initialized during index creation"); - } finally { + if (this.dependencyGraph != null) { + context.setDependencyGraph(dependencyGraph); } + for (int i = 0; i < this.iteratorSize; i++) { + CompiledIteratorDef iterDef = (CompiledIteratorDef) fromIterators.get(i); + // We are re-using the same ExecutionContext on every evaluate -- this + // is not how ExecutionContext was intended to be used. + // Asif: Compute the dependency only once. The call to methods of this + // class are thread safe as for update lock on Index is taken . + if (this.dependencyGraph == null) { + iterDef.computeDependencies(context); + } + RuntimeIterator rIter = iterDef.getRuntimeIterator(context); + context.addToIndependentRuntimeItrMapForIndexCreation(iterDef); + context.bindIterator(rIter); + } + // Save the dependency graph for future updates. + if (dependencyGraph == null) { + dependencyGraph = context.getDependencyGraph(); + } + + Support.Assert(this.indexResultSetType != null, + "IMQEvaluator::evaluate:The StructType should have been initialized during index creation"); return context; } http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexCreationHelper.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexCreationHelper.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexCreationHelper.java index 50f6811..293d22a 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexCreationHelper.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexCreationHelper.java @@ -21,81 +21,78 @@ package org.apache.geode.cache.query.internal.index; import java.util.List; -import org.apache.geode.cache.Cache; import org.apache.geode.cache.Region; -import org.apache.geode.cache.query.*; +import org.apache.geode.cache.query.IndexInvalidException; import org.apache.geode.cache.query.internal.CompiledValue; import org.apache.geode.cache.query.internal.QCompiler; +import org.apache.geode.internal.cache.InternalCache; -/** - * - */ public abstract class IndexCreationHelper { public static int INDEX_QUERY_SCOPE_ID = -2; - // Canonicalized attributes. - // The value in these fields is set during the execution of prepareFromClause - // function - // While the value of fromClause is reset in execution of prepareFromClause, - // to canonicalized from clause + + /** + * Canonicalized attributes. The value in these fields is set during the execution of + * prepareFromClause function While the value of fromClause is reset in execution of + * prepareFromClause, to canonicalized from clause + */ String fromClause; + String indexedExpression; + String projectionAttributes; - // String imports; - QCompiler compiler; // use the same compiler for each query string to use - // imports - Cache cache; - // Asif : The array containing the canonicalized iterator names - // which will get reused. - // TODO: Asif : How to make it final so that the invokers do not end up - // modifying it + + // use the same compiler for each query string to use + QCompiler compiler; + + InternalCache cache; + + /** + * The array containing the canonicalized iterator names which will get reused. + *

+ * TODO: How to make it final so that the invokers do not end up modifying it + */ String[] canonicalizedIteratorNames = null; - // Asif : Array containing canonicalized iterator definitions - // TODO: Asif : How to make it final so that the invokers do not end up - // modifying it + + /** + * Array containing canonicalized iterator definitions + *

+ * TODO: How to make it final so that the invokers do not end up modifying it + */ String[] canonicalizedIteratorDefinitions = null; - IndexCreationHelper(String fromClause, String projectionAttributes, Cache cache) + IndexCreationHelper(String fromClause, String projectionAttributes, InternalCache cache) throws IndexInvalidException { this.cache = cache; - // Asif:LThe fromClause,indexedExpression & projectionAttributes + // The fromClause,indexedExpression & projectionAttributes // will get modified with the canonicalized value , once the // constructor of derived class is over. this.fromClause = fromClause; - // this.indexedExpression = indexedExpression; this.projectionAttributes = projectionAttributes; - // this.imports = imports; - this.compiler = new QCompiler(true /* used from index creation */); - /* - * if (this.imports != null) { this.compiler.compileImports(this.imports); } - */ + this.compiler = new QCompiler(true); } public String getCanonicalizedProjectionAttributes() { - return projectionAttributes; + return this.projectionAttributes; } - /* - * TODO:Asif . Check if this function is required public String getImports() { return - * this.imports; } - */ public String getCanonicalizedIndexedExpression() { - return indexedExpression; + return this.indexedExpression; } public String getCanonicalizedFromClause() { - return fromClause; + return this.fromClause; } - public Cache getCache() { - return cache; + public InternalCache getCache() { + return this.cache; } - /* - * Asif: This function returns the canonicalized Iterator Definitions of the from clauses used in - * Index creation + /** + * This function returns the canonicalized Iterator Definitions of the from clauses used in Index + * creation */ - public String[] getCanonicalizedIteratorDefinitions() { + String[] getCanonicalizedIteratorDefinitions() { return this.canonicalizedIteratorDefinitions; } http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java index ed4f47f..2433178 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexManager.java @@ -12,33 +12,70 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ -/* - * IndexManager.java - * - * Created on February 15, 2005, 11:49 AM - */ package org.apache.geode.cache.query.internal.index; +import static org.apache.geode.internal.lang.SystemUtils.getLineSeparator; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.FutureTask; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.logging.log4j.Logger; + import org.apache.geode.SystemFailure; import org.apache.geode.cache.Region; -import org.apache.geode.cache.query.*; -import org.apache.geode.cache.query.internal.*; +import org.apache.geode.cache.query.AmbiguousNameException; +import org.apache.geode.cache.query.Index; +import org.apache.geode.cache.query.IndexExistsException; +import org.apache.geode.cache.query.IndexInvalidException; +import org.apache.geode.cache.query.IndexMaintenanceException; +import org.apache.geode.cache.query.IndexNameConflictException; +import org.apache.geode.cache.query.IndexStatistics; +import org.apache.geode.cache.query.IndexType; +import org.apache.geode.cache.query.MultiIndexCreationException; +import org.apache.geode.cache.query.NameResolutionException; +import org.apache.geode.cache.query.QueryException; +import org.apache.geode.cache.query.TypeMismatchException; +import org.apache.geode.cache.query.internal.CompiledPath; +import org.apache.geode.cache.query.internal.CompiledValue; +import org.apache.geode.cache.query.internal.DefaultQuery; +import org.apache.geode.cache.query.internal.ExecutionContext; +import org.apache.geode.cache.query.internal.MapIndexable; +import org.apache.geode.cache.query.internal.NullToken; +import org.apache.geode.cache.query.internal.QueryMonitor; +import org.apache.geode.cache.query.internal.QueryObserver; +import org.apache.geode.cache.query.internal.QueryObserverHolder; import org.apache.geode.cache.query.internal.index.AbstractIndex.InternalIndexStatistics; import org.apache.geode.cache.query.internal.parse.OQLLexerTokenTypes; import org.apache.geode.distributed.internal.DistributionConfig; import org.apache.geode.internal.Assert; -import org.apache.geode.internal.cache.*; +import org.apache.geode.internal.cache.BucketRegion; +import org.apache.geode.internal.cache.CachePerfStats; +import org.apache.geode.internal.cache.HasCachePerfStats; +import org.apache.geode.internal.cache.InternalCache; +import org.apache.geode.internal.cache.LocalRegion; +import org.apache.geode.internal.cache.PartitionedRegion; +import org.apache.geode.internal.cache.RegionEntry; +import org.apache.geode.internal.cache.TXManagerImpl; +import org.apache.geode.internal.cache.TXStateProxy; import org.apache.geode.internal.i18n.LocalizedStrings; import org.apache.geode.internal.logging.LogService; import org.apache.geode.internal.logging.LoggingThreadGroup; -import org.apache.logging.log4j.Logger; - -import java.util.*; -import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicLong; -/** - */ public class IndexManager { private static final Logger logger = LogService.getLogger(); @@ -69,8 +106,7 @@ public class IndexManager { // Threshold for Queue. private final int INDEX_MAINTENANCE_BUFFER = - Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "AsynchIndexMaintenanceThreshold", -1) - .intValue(); + Integer.getInteger(DistributionConfig.GEMFIRE_PREFIX + "AsynchIndexMaintenanceThreshold", -1); public static boolean JOIN_OPTIMIZATION = !Boolean.getBoolean(DistributionConfig.GEMFIRE_PREFIX + "index.DisableJoinOptimization"); @@ -91,18 +127,14 @@ public class IndexManager { * without this the old-values are not removed from the index-maps thus resulting in inconsistent * results. */ - public static final boolean INPLACE_OBJECT_MODIFICATION = Boolean - .valueOf(System.getProperty( - DistributionConfig.GEMFIRE_PREFIX + "index.INPLACE_OBJECT_MODIFICATION", "false")) - .booleanValue(); + public static final boolean INPLACE_OBJECT_MODIFICATION = Boolean.valueOf(System.getProperty( + DistributionConfig.GEMFIRE_PREFIX + "index.INPLACE_OBJECT_MODIFICATION", "false")); /** * System property to turn-off the compact-index support. */ - public static final boolean RANGEINDEX_ONLY = Boolean - .valueOf( - System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "index.RANGEINDEX_ONLY", "false")) - .booleanValue(); + public static final boolean RANGEINDEX_ONLY = Boolean.valueOf( + System.getProperty(DistributionConfig.GEMFIRE_PREFIX + "index.RANGEINDEX_ONLY", "false")); /** For test purpose only */ public static boolean TEST_RANGEINDEX_ONLY = false; @@ -149,7 +181,6 @@ public class IndexManager { * This is a fix for #47475 * * @param operationTime the last modified time from version tag - * @param currentCacheTime */ public static boolean setIndexBufferTime(long operationTime, long currentCacheTime) { long timeDifference = currentCacheTime - operationTime; @@ -179,20 +210,12 @@ public class IndexManager { * underneath could change. * * Small amounts of false positives are ok as it will have a slight impact on performance - * - * @param queryStartTime - * @param lastModifiedTime */ public static boolean needsRecalculation(long queryStartTime, long lastModifiedTime) { return ENABLE_UPDATE_IN_PROGRESS_INDEX_CALCULATION && queryStartTime <= SAFE_QUERY_TIME.get() - queryStartTime + lastModifiedTime; } - /** - * - * @param value - * @param newValue - */ private static boolean setNewLargestValue(AtomicLong value, long newValue) { boolean done = false; while (!done) { @@ -253,7 +276,7 @@ public class IndexManager { DefaultQuery.setPdxReadSerialized(this.region.getCache(), true); TXStateProxy tx = null; - if (!((GemFireCacheImpl) this.region.getCache()).isClient()) { + if (!((InternalCache) this.region.getCache()).isClient()) { tx = ((TXManagerImpl) this.region.getCache().getCacheTransactionManager()).internalSuspend(); } @@ -278,13 +301,14 @@ public class IndexManager { } if (indexType != IndexType.PRIMARY_KEY) { helper = new FunctionalIndexCreationHelper(origFromClause, origIndexedExpression, - projectionAttributes, imports, region.getCache(), externalContext, this); + projectionAttributes, imports, (InternalCache) region.getCache(), externalContext, + this); // Asif: For now support Map index as non compact .expand later // The limitation for compact range index also apply to hash index for now isCompactOrHash = shouldCreateCompactIndex((FunctionalIndexCreationHelper) helper); } else if (indexType == IndexType.PRIMARY_KEY) { helper = new PrimaryKeyIndexCreationHelper(origFromClause, origIndexedExpression, - projectionAttributes, region.getCache(), externalContext, this); + projectionAttributes, (InternalCache) region.getCache(), externalContext, this); } else { throw new AssertionError("Don't know how to set helper for " + indexType); } @@ -356,7 +380,7 @@ public class IndexManager { if (!(oldIndex instanceof Index)) { // Some other thread is creating the same index. // Wait for index to be initialized from other thread. - ((FutureTask) oldIndex).get(); + ((Future) oldIndex).get(); } // The Index is successfully created, throw appropriate error message @@ -370,7 +394,7 @@ public class IndexManager { LocalizedStrings.IndexManager_SIMILAR_INDEX_EXISTS.toLocalizedString()); } } - } catch (InterruptedException ie) { + } catch (InterruptedException ignored) { interrupted = true; } catch (ExecutionException ee) { Throwable c = ee.getCause(); @@ -504,9 +528,6 @@ public class IndexManager { * @param indexedExpression the indexedExpression for the index * @param context ExecutionContext * @return the sole index of the region with these parameters, or null if there isn't one - * @throws NameResolutionException - * @throws TypeMismatchException - * @throws AmbiguousNameException */ public IndexData getIndex(IndexType indexType, String[] definitions, CompiledValue indexedExpression, ExecutionContext context) @@ -514,7 +535,7 @@ public class IndexManager { IndexData indxData = null; int qItrSize = definitions.length; Iterator it = this.indexes.values().iterator(); - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); indexedExpression.generateCanonicalizedExpression(sb, context); String indexExprStr = sb.toString(); while (it.hasNext()) { @@ -569,9 +590,6 @@ public class IndexManager { * created * @param context ExecutionContext object * @return IndexData object - * @throws NameResolutionException - * @throws TypeMismatchException - * @throws AmbiguousNameException */ public IndexData getBestMatchIndex(IndexType indexType, String[] definitions, CompiledValue indexedExpression, ExecutionContext context) @@ -584,7 +602,7 @@ public class IndexManager { int qItrSize = definitions.length; int bestIndexMatchLevel = qItrSize; Iterator iter = this.indexes.values().iterator(); - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); indexedExpression.generateCanonicalizedExpression(sb, context); String indexExprStr = sb.toString(); PartitionedIndex prIndex = null; @@ -632,7 +650,7 @@ public class IndexManager { } prIndex.verifyAndCreateMissingIndex(context.getBucketList()); - } catch (Exception ex) { + } catch (Exception ignored) { // Index is not there on all buckets. // ignore this index. prIndex.releaseIndexReadLockForRemove(); @@ -784,9 +802,8 @@ public class IndexManager { Index index = (Index) ind; // Check if indexType needs to be matched. - if (indexType == null) { // No type check. - list.add(index); - } else if (index.getType() == indexType) { + if (indexType == null || index.getType() == indexType) { + // No type check. list.add(index); } } @@ -802,7 +819,6 @@ public class IndexManager { return getIndexes(null); } - // @todo need more specific list of exceptions /** * Remove the specified index. * @@ -858,8 +874,6 @@ public class IndexManager { * Index Initialization query on the region & before doing this it makes theexisting data maps * null. This is needed so that index does not miss any entry being put in the region when the * Region.clear is in progress - * - * @throws QueryException */ public void rerunIndexCreationQuery() throws QueryException { try { @@ -954,7 +968,6 @@ public class IndexManager { updateIndexes(entry, action, opCode, false); } - // @todo need more specific list of exceptions /** * Callback for IndexManager to update indexes Called from AbstractRegionMap. * @@ -962,7 +975,6 @@ public class IndexManager { * @param action action to be taken (IndexManager.ADD_ENTRY, IndexManager.UPDATE_ENTRY, * IndexManager.REMOVE_ENTRY) * @param opCode one of IndexProtocol.OTHER_OP, BEFORE_UPDATE_OP, AFTER_UPDATE_OP. - * @throws org.apache.geode.cache.query.IndexMaintenanceException */ public void updateIndexes(RegionEntry entry, int action, int opCode, boolean isDiskRecoveryInProgress) throws QueryException { @@ -992,7 +1004,7 @@ public class IndexManager { final long startPA = getCachePerfStats().startIndexUpdate(); DefaultQuery.setPdxReadSerialized(this.region.getCache(), true); TXStateProxy tx = null; - if (!((GemFireCacheImpl) this.region.getCache()).isClient()) { + if (!((InternalCache) this.region.getCache()).isClient()) { tx = ((TXManagerImpl) this.region.getCache().getCacheTransactionManager()).internalSuspend(); } @@ -1084,7 +1096,7 @@ public class IndexManager { if (((AbstractIndex) index).isPopulated() && index.getType() != IndexType.PRIMARY_KEY) { if (logger.isDebugEnabled()) { - logger.debug("Updating index: {}{} value: ", index.getName(), + logger.debug("Updating index: {}{} value: {}", index.getName(), this.region.getFullPath(), entry.getKey()); } start = ((AbstractIndex) index).updateIndexUpdateStats(); @@ -1154,7 +1166,7 @@ public class IndexManager { boolean interrupted = Thread.interrupted(); try { indexes.wait(); - } catch (InterruptedException ie) { + } catch (InterruptedException ignored) { interrupted = true; } finally { if (interrupted) { @@ -1233,7 +1245,7 @@ public class IndexManager { boolean interrupted = Thread.interrupted(); try { this.indexes.wait(); - } catch (InterruptedException ie) { + } catch (InterruptedException ignored) { interrupted = true; } finally { if (interrupted) { @@ -1263,7 +1275,7 @@ public class IndexManager { } private CachePerfStats getCachePerfStats() { - return ((LocalRegion) this.region).getCachePerfStats(); + return ((HasCachePerfStats) this.region).getCachePerfStats(); } /** @@ -1280,7 +1292,6 @@ public class IndexManager { * {@link PartitionedIndex}. * * @param prRegion the partition region that this bucket belongs to - * @throws QueryException */ public void removeBucketIndexes(PartitionedRegion prRegion) throws QueryException { IndexManager parentManager = prRegion.getIndexManager(); @@ -1298,7 +1309,7 @@ public class IndexManager { @Override public String toString() { - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); Iterator iter = this.indexes.values().iterator(); while (iter.hasNext()) { Object ind = iter.next(); @@ -1307,7 +1318,7 @@ public class IndexManager { if (ind instanceof FutureTask) { continue; } - sb.append(((Index) ind).toString()).append('\n'); + sb.append(ind).append(getLineSeparator()); } return sb.toString(); } @@ -1340,7 +1351,7 @@ public class IndexManager { String str = null; synchronized (canonicalizedIteratorNameMap) { if ((str = (String) this.canonicalizedIteratorNameMap.get(definition)) == null) { - str = new StringBuffer("index_iter").append(this.getIncrementedCounter()).toString(); + str = new StringBuilder("index_iter").append(this.getIncrementedCounter()).toString(); String temp; if ((temp = (String) this.canonicalizedIteratorNameMap.putIfAbsent(definition, str)) != null) { @@ -1365,7 +1376,6 @@ public class IndexManager { * Asif : Given a definition returns the canonicalized iterator name for the definition. If the * definition does not exist , null is returned * - * @param definition * @return String */ public String getCanonicalizedIteratorName(String definition) { @@ -1384,9 +1394,6 @@ public class IndexManager { /** * Creates instance of IndexUpdaterThread - * - * @param updateThreshold - * @param threadName */ IndexUpdaterThread(ThreadGroup group, int updateThreshold, String threadName) { super(group, threadName); @@ -1403,9 +1410,9 @@ public class IndexManager { public void addTask(int action, RegionEntry entry, int opCode) { Object[] task = new Object[3]; - task[0] = Integer.valueOf(action); + task[0] = action; task[1] = entry; - task[2] = Integer.valueOf(opCode); // !!!:ezoerner:20081029 change to valueOf jdk 1.5+ + task[2] = opCode; pendingTasks.add(task); } @@ -1454,9 +1461,9 @@ public class IndexManager { } private void updateIndexes(Object[] task) { - int action = ((Integer) task[0]).intValue(); + int action = (Integer) task[0]; RegionEntry entry = (RegionEntry) task[1]; - int opCode = ((Integer) task[2]).intValue(); + int opCode = (Integer) task[2]; // System.out.println("entry = "+entry.getKey()); if (entry != null || action == RECREATE_INDEX) { try { @@ -1535,6 +1542,7 @@ public class IndexManager { @Override public boolean equals(Object other) { + // TODO: equals should check the class of its parameter if (other == null) { return false; } @@ -1549,6 +1557,7 @@ public class IndexManager { } String[] indexDefinitions = this.helper.getCanonicalizedIteratorDefinitions(); + // TODO: avoid object creation in equals int[] mapping = new int[indexDefinitions.length]; // compare index based on its type, expression and definition. if (compareIndexData(this.indexType, indexDefinitions, http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java index a83cb9b..e1f4fa7 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexStats.java @@ -12,21 +12,21 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ - package org.apache.geode.cache.query.internal.index; -import org.apache.geode.*; -// import org.apache.geode.cache.query.*; -import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl; +import org.apache.geode.StatisticDescriptor; +import org.apache.geode.Statistics; +import org.apache.geode.StatisticsFactory; +import org.apache.geode.StatisticsType; +import org.apache.geode.StatisticsTypeFactory; import org.apache.geode.internal.cache.CachePerfStats; +import org.apache.geode.internal.statistics.StatisticsTypeFactoryImpl; /** * IndexStats tracks statistics about query index use. */ public class IndexStats { - ////////////////// Static fields /////////////////////////// - private static final StatisticsType type; private static final int numKeysId; @@ -44,8 +44,6 @@ public class IndexStats { /** The Statistics object that we delegate most behavior to */ private final Statistics stats; - //////////////////////// Static methods //////////////////////// - static { StatisticsTypeFactory f = StatisticsTypeFactoryImpl.singleton(); @@ -84,8 +82,6 @@ public class IndexStats { numBucketIndexesId = type.nameToId("numBucketIndexes"); } - //////////////////////// Constructors //////////////////////// - /** * Creates a new CachePerfStats and registers itself with the given statistics * factory. @@ -94,8 +90,6 @@ public class IndexStats { stats = factory.createAtomicStatistics(type, indexName); } - ////////////////////// Accessing Stats ////////////////////// - public long getNumberOfKeys() { return stats.getLong(numKeysId); } @@ -132,8 +126,6 @@ public class IndexStats { return stats.getInt(numBucketIndexesId); } - ////////////////////// Updating Stats ////////////////////// - public void incNumUpdates() { this.stats.incLong(numUpdatesId, 1); } @@ -189,11 +181,10 @@ public class IndexStats { public void incNumBucketIndexes(int delta) { this.stats.incInt(numBucketIndexesId, delta); } - ////// Special Instance Methods ///// /** - * Closes these stats so that they can not longer be used. The stats are closed when the - * {@linkplain org.apache.geode.internal.cache.GemFireCacheImpl#close cache} is closed. + * Closes these stats so that they can not longer be used. The stats are closed when the cache is + * closed. * * @since GemFire 3.5 */ @@ -210,7 +201,6 @@ public class IndexStats { return this.stats.isClosed(); } - /** * Returns the Statistics instance that stores the cache perf stats. * @@ -220,4 +210,3 @@ public class IndexStats { return this.stats; } } - http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexUtils.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexUtils.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexUtils.java index a6c5ec7..3bd41e0 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexUtils.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/IndexUtils.java @@ -12,30 +12,25 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ -/* - * IndexUtils.java - * - * Created on March 4, 2005, 5:39 PM - */ package org.apache.geode.cache.query.internal.index; -import org.apache.geode.cache.Cache; import org.apache.geode.cache.Region; import org.apache.geode.cache.query.AmbiguousNameException; import org.apache.geode.cache.query.IndexType; import org.apache.geode.cache.query.NameResolutionException; import org.apache.geode.cache.query.TypeMismatchException; -import org.apache.geode.internal.cache.GemFireCacheImpl; +import org.apache.geode.cache.query.internal.CompiledValue; +import org.apache.geode.cache.query.internal.DefaultQueryService; +import org.apache.geode.cache.query.internal.ExecutionContext; +import org.apache.geode.cache.query.internal.index.IndexManager.TestHook; +import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.LocalRegion; import org.apache.geode.internal.cache.PartitionedRegion; -import org.apache.geode.cache.query.internal.*; -import org.apache.geode.cache.query.internal.index.IndexManager.TestHook; -/** - */ public class IndexUtils { public static final boolean indexesEnabled = System.getProperty("query.disableIndexes") == null; + public static final boolean useOnlyExactIndexs = false; public static TestHook testHook; @@ -45,8 +40,10 @@ public class IndexUtils { } public static IndexManager getIndexManager(Region region, boolean createIfNotAvailable) { - if (region == null || region.isDestroyed()) + if (region == null || region.isDestroyed()) { return null; + } + LocalRegion lRegion = (LocalRegion) region; IndexManager idxMgr = lRegion.getIndexManager(); if (idxMgr == null && createIfNotAvailable) { @@ -68,13 +65,12 @@ public class IndexUtils { return idxMgr; } - public static IndexData findIndex(String regionpath, String defintions[], - CompiledValue indexedExpression, String projectionAttributes, Cache cache, + public static IndexData findIndex(String regionpath, String[] defintions, + CompiledValue indexedExpression, String projectionAttributes, InternalCache cache, boolean usePrimaryIndex, ExecutionContext context) throws AmbiguousNameException, TypeMismatchException, NameResolutionException { - DefaultQueryService qs = - (DefaultQueryService) ((GemFireCacheImpl) cache).getLocalQueryService(); - // IndexProtocol index = null; + + DefaultQueryService qs = (DefaultQueryService) cache.getLocalQueryService(); IndexData indxData = null; if (usePrimaryIndex) { if (useOnlyExactIndexs) { @@ -98,7 +94,6 @@ public class IndexUtils { } } - // If Primary Key Index not found or is not valid search for FUNCTIONAL // Index if (indxData == null || !indxData._index.isValid()) { @@ -110,16 +105,11 @@ public class IndexUtils { indexedExpression, context); } } else { - // if exact PRIMARY_KEY Index not found then try to find exact FUNCTIONAL - // Index - // if (!fromClause.equals(index.getCanonicalizedFromClause())) { + // if exact PRIMARY_KEY Index not found then try to find exact FUNCTIONAL Index if (indxData._matchLevel != 0) { IndexData functionalIndxData = qs.getIndex(regionpath, defintions, IndexType.FUNCTIONAL /* do not use pk index */, indexedExpression, context); - // if FUNCTIONAL Index is exact match then use or else use PRIMARY_KEY - // Index - // if (functionalIndxInfo != null && - // fromClause.equals(functionalIndex.getCanonicalizedFromClause()) + // if FUNCTIONAL Index is exact match then use or else use PRIMARY_KEY Index if (functionalIndxData != null && functionalIndxData._index.isValid()) { indxData = functionalIndxData; } http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/MemoryIndexStore.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/MemoryIndexStore.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/MemoryIndexStore.java index e9cd070..6b07490 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/MemoryIndexStore.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/MemoryIndexStore.java @@ -14,11 +14,11 @@ */ package org.apache.geode.cache.query.internal.index; -import java.util.AbstractMap.SimpleImmutableEntry; import java.util.Collection; import java.util.HashSet; import java.util.Iterator; import java.util.Map; +import java.util.Map.Entry; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -26,7 +26,6 @@ import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; import java.util.concurrent.atomic.AtomicInteger; -import org.apache.geode.cache.Cache; import org.apache.geode.cache.EntryDestroyedException; import org.apache.geode.cache.Region; import org.apache.geode.cache.RegionAttributes; @@ -40,6 +39,7 @@ import org.apache.geode.cache.query.internal.parse.OQLLexerTokenTypes; import org.apache.geode.cache.query.internal.types.TypeUtils; import org.apache.geode.internal.cache.CachedDeserializable; import org.apache.geode.internal.cache.GemFireCacheImpl; +import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.LocalRegion; import org.apache.geode.internal.cache.RegionEntry; import org.apache.geode.internal.cache.Token; @@ -59,16 +59,19 @@ public class MemoryIndexStore implements IndexStore { new ConcurrentSkipListMap(TypeUtils.getExtendedNumericComparator()); // number of keys - protected volatile AtomicInteger numIndexKeys = new AtomicInteger(0); + private final AtomicInteger numIndexKeys = new AtomicInteger(0); // Map for RegionEntries=>value of indexedExpression (reverse map) private ConcurrentMap entryToValuesMap; - private InternalIndexStatistics internalIndexStats; + private final InternalIndexStatistics internalIndexStats; + + private final InternalCache cache; + + private final Region region; - private Cache cache; - private Region region; private boolean indexOnRegionKeys; + private boolean indexOnValues; // Used as a place holder for an indexkey collection for when a thread is about to change @@ -78,13 +81,14 @@ public class MemoryIndexStore implements IndexStore { // while the other would execute a remove on the index elem. // both would complete but the remove would have been lost because we had already added it to the // new collection - private Object TRANSITIONING_TOKEN = new IndexElemArray(1); + private final Object TRANSITIONING_TOKEN = new IndexElemArray(1); - public MemoryIndexStore(Region region, InternalIndexStatistics internalIndexStats) { + MemoryIndexStore(Region region, InternalIndexStatistics internalIndexStats) { this(region, internalIndexStats, GemFireCacheImpl.getInstance()); } - public MemoryIndexStore(Region region, InternalIndexStatistics internalIndexStats, Cache cache) { + private MemoryIndexStore(Region region, InternalIndexStatistics internalIndexStats, + InternalCache cache) { this.region = region; RegionAttributes ra = region.getAttributes(); // Initialize the reverse-map if in-place modification is set by the @@ -98,7 +102,7 @@ public class MemoryIndexStore implements IndexStore { } @Override - public void updateMapping(Object newKey, Object oldKey, RegionEntry entry, Object oldValue) + public void updateMapping(Object indexKey, Object oldKey, RegionEntry re, Object oldValue) throws IMQException { try { @@ -109,38 +113,39 @@ public class MemoryIndexStore implements IndexStore { // Check if reverse-map is present. if (IndexManager.isObjectModificationInplace()) { // If reverse map get the old index key from reverse map. - if (this.entryToValuesMap.containsKey(entry)) { - oldKey = this.entryToValuesMap.get(entry); + if (this.entryToValuesMap.containsKey(re)) { + oldKey = this.entryToValuesMap.get(re); } } else { // Check if the old value and new value same. // If they are same, that means the value got updated in place. // In the absence of reverse-map find the old index key from // forward map. - if (oldValue != null && oldValue == getTargetObjectInVM(entry)) { - oldKey = getOldKey(newKey, entry); + if (oldValue != null && oldValue == getTargetObjectInVM(re)) { + oldKey = getOldKey(indexKey, re); } } // No need to update the map if new and old index key are same. - if (oldKey != null && oldKey.equals(TypeUtils.indexKeyFor(newKey))) { + if (oldKey != null && oldKey.equals(TypeUtils.indexKeyFor(indexKey))) { return; } boolean retry = false; - newKey = TypeUtils.indexKeyFor(newKey); - if (newKey.equals(QueryService.UNDEFINED)) { - Object targetObject = getTargetObjectForUpdate(entry); + indexKey = TypeUtils.indexKeyFor(indexKey); + if (indexKey.equals(QueryService.UNDEFINED)) { + Object targetObject = getTargetObjectForUpdate(re); if (Token.isInvalidOrRemoved(targetObject)) { if (oldKey != null) { - basicRemoveMapping(oldKey, entry, false); + basicRemoveMapping(oldKey, re, false); } return; } } + do { retry = false; - Object regionEntries = this.valueToEntriesMap.putIfAbsent(newKey, entry); + Object regionEntries = this.valueToEntriesMap.putIfAbsent(indexKey, re); if (regionEntries == TRANSITIONING_TOKEN) { retry = true; continue; @@ -153,8 +158,8 @@ public class MemoryIndexStore implements IndexStore { DefaultQuery.testHook.doTestHook("BEGIN_TRANSITION_FROM_REGION_ENTRY_TO_ELEMARRAY"); } elemArray.add(regionEntries); - elemArray.add(entry); - if (!this.valueToEntriesMap.replace(newKey, regionEntries, elemArray)) { + elemArray.add(re); + if (!this.valueToEntriesMap.replace(indexKey, regionEntries, elemArray)) { retry = true; } if (DefaultQuery.testHook != null) { @@ -168,9 +173,9 @@ public class MemoryIndexStore implements IndexStore { // ConcurrentHashSet when set size becomes zero during // basicRemoveMapping(); synchronized (regionEntries) { - ((IndexConcurrentHashSet) regionEntries).add(entry); + ((IndexConcurrentHashSet) regionEntries).add(re); } - if (regionEntries != this.valueToEntriesMap.get(newKey)) { + if (regionEntries != this.valueToEntriesMap.get(indexKey)) { retry = true; } } else { @@ -192,15 +197,15 @@ public class MemoryIndexStore implements IndexStore { // captured // by our instance of the elem array, or the remove operations will need to do a // retry? - if (!this.valueToEntriesMap.replace(newKey, regionEntries, TRANSITIONING_TOKEN)) { + if (!this.valueToEntriesMap.replace(indexKey, regionEntries, TRANSITIONING_TOKEN)) { retry = true; } else { if (DefaultQuery.testHook != null) { DefaultQuery.testHook.doTestHook("TRANSITIONED_FROM_ELEMARRAY_TO_TOKEN"); } - set.add(entry); + set.add(re); set.addAll(elemArray); - if (!this.valueToEntriesMap.replace(newKey, TRANSITIONING_TOKEN, set)) { + if (!this.valueToEntriesMap.replace(indexKey, TRANSITIONING_TOKEN, set)) { // This should never happen. If we see this in the log, then something is wrong // with the TRANSITIONING TOKEN and synchronization of changing collection types // we should then just go from RE to CHS and completely remove the Elem Array. @@ -215,8 +220,8 @@ public class MemoryIndexStore implements IndexStore { } } } else { - elemArray.add(entry); - if (regionEntries != this.valueToEntriesMap.get(newKey)) { + elemArray.add(re); + if (regionEntries != this.valueToEntriesMap.get(indexKey)) { retry = true; } } @@ -229,16 +234,16 @@ public class MemoryIndexStore implements IndexStore { // remove from forward map in case of update // oldKey is not null only for an update if (oldKey != null) { - basicRemoveMapping(oldKey, entry, false); + basicRemoveMapping(oldKey, re, false); } if (IndexManager.isObjectModificationInplace()) { - this.entryToValuesMap.put(entry, newKey); + this.entryToValuesMap.put(re, indexKey); } } } while (retry); } catch (TypeMismatchException ex) { - throw new IMQException("Could not add object of type " + newKey.getClass().getName(), ex); + throw new IMQException("Could not add object of type " + indexKey.getClass().getName(), ex); } internalIndexStats.incNumValues(1); } @@ -251,8 +256,8 @@ public class MemoryIndexStore implements IndexStore { */ private Object getOldKey(Object newKey, RegionEntry entry) throws TypeMismatchException { for (Object mapEntry : valueToEntriesMap.entrySet()) { - Object regionEntries = ((SimpleImmutableEntry) mapEntry).getValue(); - Object indexKey = ((SimpleImmutableEntry) mapEntry).getKey(); + Object regionEntries = ((Entry) mapEntry).getValue(); + Object indexKey = ((Entry) mapEntry).getKey(); // if more than one index key maps to the same RegionEntry that // means there has been an in-place modification if (TypeUtils.compare(indexKey, newKey, CompiledComparison.TOK_NE).equals(Boolean.TRUE)) { @@ -270,34 +275,34 @@ public class MemoryIndexStore implements IndexStore { } @Override - public void addMapping(Object newKey, RegionEntry entry) throws IMQException { + public void addMapping(Object indexKey, RegionEntry re) throws IMQException { // for add, oldkey is null - updateMapping(newKey, null, entry, null); + updateMapping(indexKey, null, re, null); } @Override - public void removeMapping(Object key, RegionEntry entry) throws IMQException { + public void removeMapping(Object indexKey, RegionEntry re) throws IMQException { // Remove from forward map - boolean found = basicRemoveMapping(key, entry, true); + boolean found = basicRemoveMapping(indexKey, re, true); // Remove from reverse map. // We do NOT need to synchronize here as different RegionEntries will be // operating concurrently i.e. different keys in entryToValuesMap which // is a concurrent map. if (found && IndexManager.isObjectModificationInplace()) { - this.entryToValuesMap.remove(entry); + this.entryToValuesMap.remove(re); } } - protected boolean basicRemoveMapping(Object key, RegionEntry entry, boolean findOldKey) + private boolean basicRemoveMapping(Object key, RegionEntry entry, boolean findOldKey) throws IMQException { boolean found = false; boolean possiblyAlreadyRemoved = false; try { - boolean retry = false; Object newKey = convertToIndexKey(key, entry); if (DefaultQuery.testHook != null) { DefaultQuery.testHook.doTestHook("ATTEMPT_REMOVE"); } + boolean retry = false; do { retry = false; Object regionEntries = this.valueToEntriesMap.get(newKey); @@ -309,7 +314,7 @@ public class MemoryIndexStore implements IndexStore { continue; } else if (regionEntries != null) { if (regionEntries instanceof RegionEntry) { - found = (regionEntries == entry); + found = regionEntries == entry; if (found) { if (this.valueToEntriesMap.remove(newKey, regionEntries)) { numIndexKeys.decrementAndGet(); @@ -363,7 +368,7 @@ public class MemoryIndexStore implements IndexStore { if (found) { // Update stats if entry was actually removed internalIndexStats.incNumValues(-1); - } else if ((!found && !possiblyAlreadyRemoved) && !IndexManager.isObjectModificationInplace() + } else if (!found && !possiblyAlreadyRemoved && !IndexManager.isObjectModificationInplace() && key != null) { // if there is an inplace-modification find old key by iterating // over fwd map and then remove the mapping @@ -375,12 +380,6 @@ public class MemoryIndexStore implements IndexStore { throw new IMQException("Could not find old key: " + key.getClass().getName(), e); } } - // The entry might have been already removed by other thread - // if still not found - // if (!found) { - // throw new IMQException("index maintenance error: " - // + "entry not found for " + key + " entry: " + entry); - // } } return found; } @@ -395,14 +394,6 @@ public class MemoryIndexStore implements IndexStore { return newKey; } - /** - * Convert a RegionEntry or THashSet to be consistently a Collection - */ - /* - * private Collection regionEntryCollection(Object regionEntries) { if (regionEntries instanceof - * RegionEntry) { return Collections.singleton(regionEntries); } return (Collection) - * regionEntries; } - */ @Override public CloseableIterator get(Object indexKey) { return new MemoryIndexStoreIterator( @@ -492,7 +483,7 @@ public class MemoryIndexStore implements IndexStore { if (o instanceof CachedDeserializable) { return ((CachedDeserializable) o).getDeserializedValue(this.region, entry); } - } catch (EntryDestroyedException ede) { + } catch (EntryDestroyedException ignore) { return null; } return o; @@ -502,6 +493,7 @@ public class MemoryIndexStore implements IndexStore { return new CachedEntryWrapper(((LocalRegion) this.region).new NonTXEntry(entry)); } + @Override public Object getTargetObjectInVM(RegionEntry entry) { if (indexOnValues) { Object o = entry.getValueInVM((LocalRegion) this.region); @@ -522,7 +514,7 @@ public class MemoryIndexStore implements IndexStore { return ((LocalRegion) this.region).new NonTXEntry(entry); } - public Object getTargetObjectForUpdate(RegionEntry entry) { + private Object getTargetObjectForUpdate(RegionEntry entry) { if (indexOnValues) { Object o = entry.getValue((LocalRegion) this.region); try { @@ -556,7 +548,7 @@ public class MemoryIndexStore implements IndexStore { public int size(Object key) { Object obj = valueToEntriesMap.get(key); if (obj != null) { - return (obj instanceof RegionEntry) ? 1 : ((Collection) obj).size(); + return obj instanceof RegionEntry ? 1 : ((Collection) obj).size(); } else { return 0; } @@ -575,15 +567,14 @@ public class MemoryIndexStore implements IndexStore { final Map map; Object indexKey; Collection keysToRemove; - - protected Iterator mapIterator; - protected Iterator valuesIterator; - protected Object currKey; - protected Object currValue; // RegionEntry + Iterator mapIterator; + Iterator valuesIterator; + Object currKey; + Object currValue; // RegionEntry final long iteratorStartTime; - protected MemoryIndexStoreEntry currentEntry; + MemoryIndexStoreEntry currentEntry; - private MemoryIndexStoreIterator(Map submap, Object indexKey, Collection keysToRemove) { + MemoryIndexStoreIterator(Map submap, Object indexKey, Collection keysToRemove) { this(submap, indexKey, keysToRemove, GemFireCacheImpl.getInstance().cacheTimeMillis()); } http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PartitionedIndex.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PartitionedIndex.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PartitionedIndex.java index 1938487..b8e7938 100755 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PartitionedIndex.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PartitionedIndex.java @@ -550,7 +550,7 @@ public class PartitionedIndex extends AbstractIndex { } @Override - void instantiateEvaluator(IndexCreationHelper ich) { + void instantiateEvaluator(IndexCreationHelper indexCreationHelper) { throw new UnsupportedOperationException(); } http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndex.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndex.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndex.java index 2db59b2..8cc036f 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndex.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndex.java @@ -12,11 +12,6 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ -/* - * PrimaryKeyIndex.java - * - * Created on March 20, 2005, 6:47 PM - */ package org.apache.geode.cache.query.internal.index; import java.util.Collection; @@ -51,9 +46,6 @@ import org.apache.geode.internal.cache.RegionEntry; import org.apache.geode.internal.i18n.LocalizedStrings; import org.apache.geode.pdx.internal.PdxString; -/** - * - */ public class PrimaryKeyIndex extends AbstractIndex { protected long numUses = 0; @@ -64,7 +56,7 @@ public class PrimaryKeyIndex extends AbstractIndex { String origIndxExpr, String[] defintions, IndexStatistics indexStatistics) { super(indexName, region, fromClause, indexedExpression, projectionAttributes, origFromClause, origIndxExpr, defintions, indexStatistics); - // TODO : Asif Check if the below is correct + // TODO: Check if the below is correct Class constr = region.getAttributes().getValueConstraint(); if (constr == null) constr = Object.class; @@ -91,21 +83,13 @@ public class PrimaryKeyIndex extends AbstractIndex { void addMapping(RegionEntry entry) throws IMQException {} @Override - void instantiateEvaluator(IndexCreationHelper ich) {} + void instantiateEvaluator(IndexCreationHelper indexCreationHelper) {} @Override void lockedQuery(Object key, int operator, Collection results, Set keysToRemove, ExecutionContext context) throws TypeMismatchException { assert keysToRemove == null; - // System.out.println("PrimaryKeyIndex.lockedQuery"); - // System.out.println(" key="+key); - // System.out.println(" key.class="+(key != null ? key.getClass().getName() - // : "null")); - // if(key == null){ - // numUses++; - // return; - // } - // key = TypeUtils.indexKeyFor(key); + int limit = -1; // Key cannot be PdxString in a region @@ -114,8 +98,8 @@ public class PrimaryKeyIndex extends AbstractIndex { } Boolean applyLimit = (Boolean) context.cacheGet(CompiledValue.CAN_APPLY_LIMIT_AT_INDEX); - if (applyLimit != null && applyLimit.booleanValue()) { - limit = ((Integer) context.cacheGet(CompiledValue.RESULT_LIMIT)).intValue(); + if (applyLimit != null && applyLimit) { + limit = (Integer) context.cacheGet(CompiledValue.RESULT_LIMIT); } QueryObserver observer = QueryObserverHolder.getInstance(); if (limit != -1 && results.size() == limit) { @@ -235,8 +219,7 @@ public class PrimaryKeyIndex extends AbstractIndex { continue; } Object val = entry.getValue(); - // TODO:Asif: is this correct. What should be the behaviour of null - // values? + // TODO: is this correct. What should be the behaviour of null values? if (val != null) { boolean ok = true; if (runtimeItr != null) { @@ -253,22 +236,6 @@ public class PrimaryKeyIndex extends AbstractIndex { } } } - // if (key != null && key != QueryService.UNDEFINED) { - // Region.Entry entry = getRegion().getEntry(key); - // if (entry != null) { - // Object val = entry.getValue(); - // if (val != null) { - // boolean ok = true; - // if (runtimeItr != null) { - // runtimeItr.setCurrent(val); - // ok = QueryUtils.applyCondition(iterOps, context); - // } - // if (ok) { - // applyProjection(projAttrib, context, results,val,intermediateResults,isIntersection); - // } - // } - // } - // } break; } default: { @@ -335,14 +302,6 @@ public class PrimaryKeyIndex extends AbstractIndex { } } - - /* - * (non-Javadoc) - * - * @see org.apache.geode.cache.query.internal.index.AbstractIndex#lockedQuery(java.lang.Object, - * int, java.lang.Object, int, java.util.Collection, java.util.Set) - */ - @Override void lockedQuery(Object lowerBoundKey, int lowerBoundOperator, Object upperBoundKey, int upperBoundOperator, Collection results, Set keysToRemove, ExecutionContext context) @@ -359,8 +318,7 @@ public class PrimaryKeyIndex extends AbstractIndex { @Override void addMapping(Object key, Object value, RegionEntry entry) throws IMQException { - // TODO Auto-generated method stub - + // do nothing } @Override @@ -368,7 +326,6 @@ public class PrimaryKeyIndex extends AbstractIndex { // Do Nothing; We are not going to call this for PrimaryKeyIndex ever. } - public boolean isEmpty() { return createStats("primaryKeyIndex").getNumberOfKeys() == 0 ? true : false; } http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndexCreationHelper.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndexCreationHelper.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndexCreationHelper.java index 558aa3b..5800fc9 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndexCreationHelper.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/PrimaryKeyIndexCreationHelper.java @@ -12,16 +12,10 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ -/* - * PrimaryIndexCreationHelper.java - * - * Created on March 20, 2005, 7:21 PM - */ package org.apache.geode.cache.query.internal.index; import java.util.List; -import org.apache.geode.cache.Cache; import org.apache.geode.cache.Region; import org.apache.geode.cache.query.IndexInvalidException; import org.apache.geode.cache.query.internal.CompiledIteratorDef; @@ -29,12 +23,10 @@ import org.apache.geode.cache.query.internal.CompiledValue; import org.apache.geode.cache.query.internal.ExecutionContext; import org.apache.geode.cache.query.internal.RuntimeIterator; import org.apache.geode.cache.query.internal.parse.OQLLexerTokenTypes; +import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.PartitionedRegion; import org.apache.geode.internal.i18n.LocalizedStrings; -/** - * - */ public class PrimaryKeyIndexCreationHelper extends IndexCreationHelper { ExecutionContext context = null; @@ -42,8 +34,8 @@ public class PrimaryKeyIndexCreationHelper extends IndexCreationHelper { final Region region; public PrimaryKeyIndexCreationHelper(String fromClause, String indexedExpression, - String projectionAttributes, Cache cache, ExecutionContext externalContext, IndexManager imgr) - throws IndexInvalidException { + String projectionAttributes, InternalCache cache, ExecutionContext externalContext, + IndexManager imgr) throws IndexInvalidException { super(fromClause, projectionAttributes, cache); if (externalContext == null) { context = new ExecutionContext(null, cache); @@ -76,7 +68,7 @@ public class PrimaryKeyIndexCreationHelper extends IndexCreationHelper { String definition = rIter.getDefinition(); this.canonicalizedIteratorDefinitions = new String[1]; this.canonicalizedIteratorDefinitions[0] = definition; - // Asif: Bind the Index_Internal_ID to the RuntimeIterator + // Bind the Index_Internal_ID to the RuntimeIterator PartitionedRegion pr = this.context.getPartitionedRegion(); this.canonicalizedIteratorNames = new String[1]; String name = null; @@ -88,7 +80,7 @@ public class PrimaryKeyIndexCreationHelper extends IndexCreationHelper { rIter.setIndexInternalID(name); this.canonicalizedIteratorNames = new String[1]; this.canonicalizedIteratorNames[0] = name; - this.fromClause = new StringBuffer(definition).append(' ').append(name).toString(); + this.fromClause = new StringBuilder(definition).append(' ').append(name).toString(); context.bindIterator(rIter); } catch (IndexInvalidException e) { throw e; // propagate @@ -110,11 +102,10 @@ public class PrimaryKeyIndexCreationHelper extends IndexCreationHelper { LocalizedStrings.PrimaryKeyIndexCreationHelper_INVALID_INDEXED_EXPRESSOION_0 .toLocalizedString(indexedExpression)); try { - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); expr.generateCanonicalizedExpression(sb, context); this.indexedExpression = sb.toString(); } catch (Exception e) { - // e.printStackTrace(); throw new IndexInvalidException( LocalizedStrings.PrimaryKeyIndexCreationHelper_INVALID_INDEXED_EXPRESSOION_0_N_1 .toLocalizedString(new Object[] {indexedExpression, e.getMessage()})); http://git-wip-us.apache.org/repos/asf/geode/blob/640b9f03/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/RangeIndex.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/RangeIndex.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/RangeIndex.java index 5ac48bc..316c30d 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/RangeIndex.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/RangeIndex.java @@ -117,8 +117,8 @@ public class RangeIndex extends AbstractIndex { } @Override - void instantiateEvaluator(IndexCreationHelper ich) { - this.evaluator = new IMQEvaluator(ich); + void instantiateEvaluator(IndexCreationHelper indexCreationHelper) { + this.evaluator = new IMQEvaluator(indexCreationHelper); } @Override @@ -992,7 +992,7 @@ public class RangeIndex extends AbstractIndex { if (entriesMap == null || result == null) return; QueryObserver observer = QueryObserverHolder.getInstance(); - if (verifyLimit(result, limit, context)) { + if (verifyLimit(result, limit)) { observer.limitAppliedAtIndexLevel(this, limit, result); return; } @@ -1010,7 +1010,7 @@ public class RangeIndex extends AbstractIndex { if (keysToRemove == null || !keysToRemove.remove(key)) { RegionEntryToValuesMap rvMap = (RegionEntryToValuesMap) entry.getValue(); rvMap.addValuesToCollection(result, limit, context); - if (verifyLimit(result, limit, context)) { + if (verifyLimit(result, limit)) { observer.limitAppliedAtIndexLevel(this, limit, result); return; } @@ -1048,7 +1048,7 @@ public class RangeIndex extends AbstractIndex { if (entriesMap == null || result == null) return; QueryObserver observer = QueryObserverHolder.getInstance(); - if (verifyLimit(result, limit, context)) { + if (verifyLimit(result, limit)) { observer.limitAppliedAtIndexLevel(this, limit, result); return; } @@ -1062,7 +1062,7 @@ public class RangeIndex extends AbstractIndex { if (foundKeyToRemove || !keyToRemove.equals(entry.getKey())) { RegionEntryToValuesMap rvMap = (RegionEntryToValuesMap) entry.getValue(); rvMap.addValuesToCollection(result, limit, context); - if (verifyLimit(result, limit, context)) { + if (verifyLimit(result, limit)) { observer.limitAppliedAtIndexLevel(this, limit, result); return; } @@ -1078,8 +1078,7 @@ public class RangeIndex extends AbstractIndex { throws FunctionDomainException, TypeMismatchException, NameResolutionException, QueryInvocationTargetException { boolean limitApplied = false; - if (entriesMap == null || result == null - || (limitApplied = verifyLimit(result, limit, context))) { + if (entriesMap == null || result == null || (limitApplied = verifyLimit(result, limit))) { if (limitApplied) { QueryObserver observer = QueryObserverHolder.getInstance(); if (observer != null) { @@ -1106,7 +1105,7 @@ public class RangeIndex extends AbstractIndex { RegionEntryToValuesMap rvMap = (RegionEntryToValuesMap) entry.getValue(); rvMap.addValuesToCollection(result, iterOps, runtimeItr, context, projAttrib, intermediateResults, isIntersection, limit); - if (verifyLimit(result, limit, context)) { + if (verifyLimit(result, limit)) { observer.limitAppliedAtIndexLevel(this, limit, result); break; } @@ -1526,10 +1525,6 @@ public class RangeIndex extends AbstractIndex { this.vsdStats.incReadLockCount(delta); } - public long getUseTime() { - return this.vsdStats.getUseTime(); - } - /** * Returns the total amount of time (in nanoseconds) spent updating this index. */