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 24BCA200C8C for ; Mon, 1 May 2017 23:21:52 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 23564160BB9; Mon, 1 May 2017 21:21:52 +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 38BF2160BD5 for ; Mon, 1 May 2017 23:21:46 +0200 (CEST) Received: (qmail 84153 invoked by uid 500); 1 May 2017 21:21:44 -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 83142 invoked by uid 99); 1 May 2017 21:21:44 -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 21:21:44 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 02C0CE9686; Mon, 1 May 2017 21:21:44 +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 21:22:10 -0000 Message-Id: <99db47a78fb346e98b331db3c534ede0@git.apache.org> In-Reply-To: <4f24f022b87b4021b6ff8c72c616e554@git.apache.org> References: <4f24f022b87b4021b6ff8c72c616e554@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [28/36] geode git commit: GEODE-2632: change dependencies on GemFireCacheImpl to InternalCache archived-at: Mon, 01 May 2017 21:21:52 -0000 http://git-wip-us.apache.org/repos/asf/geode/blob/57e93a5a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/AbstractMapIndex.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/AbstractMapIndex.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/AbstractMapIndex.java index 2956649..d7d6351 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/AbstractMapIndex.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/AbstractMapIndex.java @@ -26,6 +26,7 @@ import org.apache.geode.cache.Region; import org.apache.geode.cache.RegionAttributes; import org.apache.geode.cache.query.AmbiguousNameException; import org.apache.geode.cache.query.FunctionDomainException; +import org.apache.geode.cache.query.Index; import org.apache.geode.cache.query.IndexStatistics; import org.apache.geode.cache.query.IndexType; import org.apache.geode.cache.query.NameResolutionException; @@ -43,7 +44,7 @@ import org.apache.geode.internal.cache.BucketRegion; import org.apache.geode.internal.cache.RegionEntry; public abstract class AbstractMapIndex extends AbstractIndex { - final protected boolean isAllKeys; + final boolean isAllKeys; final String[] patternStr; @@ -187,7 +188,7 @@ public abstract class AbstractMapIndex extends AbstractIndex { public long getNumberOfValues(Object key) { long numValues = 0; for (Object ind : mapKeyToValueIndex.values()) { - numValues += ((AbstractIndex) ind).getStatistics().getNumberOfValues(key); + numValues += ((Index) ind).getStatistics().getNumberOfValues(key); } return numValues; } @@ -199,13 +200,12 @@ public abstract class AbstractMapIndex extends AbstractIndex { return this.vsdStats.getReadLockCount(); } - public void close() { this.vsdStats.close(); } public String toString() { - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); sb.append("No Keys = ").append(getNumberOfKeys()).append("\n"); sb.append("No Map Index Keys = ").append(getNumberOfMapIndexKeys()).append("\n"); sb.append("No Values = ").append(getNumberOfValues()).append("\n"); @@ -222,8 +222,8 @@ public abstract class AbstractMapIndex extends AbstractIndex { } @Override - void instantiateEvaluator(IndexCreationHelper ich) { - this.evaluator = new IMQEvaluator(ich); + void instantiateEvaluator(IndexCreationHelper indexCreationHelper) { + this.evaluator = new IMQEvaluator(indexCreationHelper); } @Override @@ -375,7 +375,7 @@ public abstract class AbstractMapIndex extends AbstractIndex { if (condnExpr instanceof MapIndexable) { MapIndexable mi = (MapIndexable) condnExpr; CompiledValue recvr = mi.getRecieverSansIndexArgs(); - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); recvr.generateCanonicalizedExpression(sb, context); sb.append('[').append(']'); return sb.toString().equals(this.patternStr[0]); http://git-wip-us.apache.org/repos/asf/geode/blob/57e93a5a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactMapRangeIndex.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactMapRangeIndex.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactMapRangeIndex.java index 3bffc9c..43aeabc 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactMapRangeIndex.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactMapRangeIndex.java @@ -46,9 +46,9 @@ public class CompactMapRangeIndex extends AbstractMapIndex { } @Override - void instantiateEvaluator(IndexCreationHelper ich) { - this.evaluator = new IMQEvaluator(ich); - this.ich = ich; + void instantiateEvaluator(IndexCreationHelper indexCreationHelper) { + this.evaluator = new IMQEvaluator(indexCreationHelper); + this.ich = indexCreationHelper; } @Override http://git-wip-us.apache.org/repos/asf/geode/blob/57e93a5a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactRangeIndex.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactRangeIndex.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactRangeIndex.java index 80568f5..139ce54 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactRangeIndex.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/CompactRangeIndex.java @@ -12,7 +12,6 @@ * 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 java.util.ArrayList; @@ -24,7 +23,8 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; -import org.apache.geode.cache.Cache; +import org.apache.logging.log4j.Logger; + import org.apache.geode.cache.CacheException; import org.apache.geode.cache.EntryDestroyedException; import org.apache.geode.cache.Region; @@ -62,15 +62,16 @@ 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.cache.query.types.StructType; +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.RegionEntryContext; import org.apache.geode.internal.cache.VMThinRegionEntryHeap; import org.apache.geode.internal.cache.persistence.query.CloseableIterator; import org.apache.geode.internal.i18n.LocalizedStrings; +import org.apache.geode.internal.logging.LogService; import org.apache.geode.pdx.internal.PdxString; -// @todo Extend to support the keys or entries of a region. /** * A CompactRangeIndex is a range index that has simple data structures to minimize its footprint, * at the expense of doing extra work at index maintenance. It is selected as the index @@ -84,6 +85,7 @@ import org.apache.geode.pdx.internal.PdxString; * @since GemFire 6.0 */ public class CompactRangeIndex extends AbstractIndex { + private static final Logger logger = LogService.getLogger(); private static TestHook testHook; @@ -194,13 +196,11 @@ public class CompactRangeIndex extends AbstractIndex { Object innerEntry = null; Object outerKey = null; Object innerKey = null; - // boolean incrementOuter = true; boolean incrementInner = true; outer: while (outer.hasNext()) { - // if (incrementOuter) { outerEntry = outer.next(); - // } outerKey = outerEntry.getDeserializedKey(); + // TODO: eliminate all labels inner: while (!incrementInner || inner.hasNext()) { if (incrementInner) { innerEntry = inner.next(); @@ -234,14 +234,14 @@ public class CompactRangeIndex extends AbstractIndex { incrementInner = true; continue outer; } else if (compare < 0) { - // Asif :The outer key is smaller than the inner key. That means + // The outer key is smaller than the inner key. That means // that we need // to increment the outer loop without moving inner loop. // incrementOuter = true; incrementInner = false; continue outer; } else { - // Asif : The outer key is greater than inner key , so increment the + // The outer key is greater than inner key , so increment the // inner loop without changing outer incrementInner = true; } @@ -269,15 +269,7 @@ public class CompactRangeIndex extends AbstractIndex { * * This method is called only for Memory indexstore * - * @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 */ protected boolean verifyInnerAndOuterEntryValues(IndexStoreEntry entry, ExecutionContext context, IndexInfo indexInfo, Object keyVal) throws FunctionDomainException, TypeMismatchException, @@ -437,7 +429,7 @@ public class CompactRangeIndex extends AbstractIndex { } break; } - } catch (EntryDestroyedException e) { + } catch (EntryDestroyedException ignore) { return Integer.MAX_VALUE; } finally { updateIndexUseEndStats(start, false); @@ -457,8 +449,8 @@ public class CompactRangeIndex 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); @@ -466,7 +458,7 @@ public class CompactRangeIndex extends AbstractIndex { 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(); @@ -494,15 +486,15 @@ public class CompactRangeIndex 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.booleanValue()) { + limit = (Integer) context.cacheGet(CompiledValue.RESULT_LIMIT); } Boolean orderByClause = (Boolean) context.cacheGet(CompiledValue.CAN_APPLY_ORDER_BY_AT_INDEX); List orderByAttrs = null; boolean asc = true; 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(); @@ -669,8 +661,8 @@ public class CompactRangeIndex extends AbstractIndex { } @Override - void instantiateEvaluator(IndexCreationHelper ich) { - this.evaluator = new IMQEvaluator(ich); + void instantiateEvaluator(IndexCreationHelper indexCreationHelper) { + this.evaluator = new IMQEvaluator(indexCreationHelper); } // Only used by CompactMapRangeIndex. This is due to the way the index initialization happens @@ -726,7 +718,7 @@ public class CompactRangeIndex extends AbstractIndex { QueryObserver observer = QueryObserverHolder.getInstance(); boolean limitApplied = false; - if (entriesIter == null || (limitApplied = verifyLimit(result, limit, context))) { + if (entriesIter == null || (limitApplied = verifyLimit(result, limit))) { if (limitApplied) { if (observer != null) { observer.limitAppliedAtIndexLevel(this, limit, result); @@ -755,7 +747,7 @@ public class CompactRangeIndex extends AbstractIndex { IndexStoreEntry indexEntry = null; try { indexEntry = entriesIter.next(); - } catch (NoSuchElementException ex) { + } catch (NoSuchElementException ignore) { // We are done with all the elements in array. // Continue from while. continue; @@ -769,48 +761,7 @@ public class CompactRangeIndex extends AbstractIndex { continue; } seenKey.add(rk); - // Some code that we might be able to use to optimize skipping the - // expansion of a value if no expansion is needed - // if - // (((CompactRangeIndex.IMQEvaluator)evaluator).getInitContext().getCurrentIterators().size() - // == 1) { - // boolean structType = (evaluator.getIndexResultSetType() instanceof - // StructType); - // if (!structType) { - // boolean ok = true; - // if (indexEntry.isUpdateInProgress()) { - // IndexInfo indexInfo = - // (IndexInfo)context.cacheGet(CompiledValue.INDEX_INFO); - // if (runtimeItr == null) { - // runtimeItr = getRuntimeIteratorForThisIndex(context, indexInfo); - // } - // runtimeItr.setCurrent(value); - // // Verify index key in region entry value. - // ok = evaluateEntry((IndexInfo) indexInfo, context, null); - // } - // if (runtimeItr != null) { - // runtimeItr.setCurrent(value); - // } - // if (ok && runtimeItr != null && iterOps != null) { - // ok = QueryUtils.applyCondition(iterOps, context); - // } - // if (ok) { - // if (context != null && context.isCqQueryContext()) { - // result.add(new CqEntry(indexEntry.getDeserializedRegionKey(), - // value)); - // } else { - // applyProjection(projAttrib, context, result, value, - // intermediateResults, isIntersection); - // } - // if (verifyLimit(result, limit, context)) { - // observer.limitAppliedAtIndexLevel(this, limit, result); - // return; - // } - // } - // continue; - // } - // } - // + List expandedResults = expandValue(context, lowerBoundKey, upperBoundKey, lowerBoundOperator, upperBoundOperator, value); Iterator iterator = ((Collection) expandedResults).iterator(); @@ -818,19 +769,7 @@ public class CompactRangeIndex extends AbstractIndex { value = iterator.next(); if (value != null) { boolean ok = true; - // We should not need to call the commented out code if expansion - // is occuring as we already reevaluate the index key per value - // if (indexEntry.isUpdateInProgress()) { - // IndexInfo indexInfo = - // (IndexInfo)context.cacheGet(CompiledValue.INDEX_INFO); - // if (runtimeItr == null) { - // runtimeItr = getRuntimeIteratorForThisIndex(context, - // indexInfo); - // } - // runtimeItr.setCurrent(value); - // // Verify index key in region entry value. - // ok = evaluateEntry((IndexInfo) indexInfo, context, null); - // } + if (runtimeItr != null) { runtimeItr.setCurrent(value); } @@ -844,7 +783,7 @@ public class CompactRangeIndex extends AbstractIndex { applyProjection(projAttrib, context, result, value, intermediateResults, isIntersection); } - if (verifyLimit(result, limit, context)) { + if (verifyLimit(result, limit)) { observer.limitAppliedAtIndexLevel(this, limit, result); return; } @@ -885,16 +824,14 @@ public class CompactRangeIndex extends AbstractIndex { applyProjection(projAttrib, context, result, value, intermediateResults, isIntersection); } - if (verifyLimit(result, limit, context)) { + if (verifyLimit(result, limit)) { observer.limitAppliedAtIndexLevel(this, limit, result); return; } } } } - } catch (ClassCastException e) { - - } catch (EntryDestroyedException e) { + } catch (ClassCastException | EntryDestroyedException ignore) { // ignore it } } @@ -908,7 +845,7 @@ public class CompactRangeIndex extends AbstractIndex { upperBoundOperator, value); return expandedResults; } catch (IMQException e) { - e.printStackTrace(); + // TODO: never throw an anonymous inner class throw new CacheException(e) {}; } } @@ -921,14 +858,7 @@ public class CompactRangeIndex extends AbstractIndex { * 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 */ protected boolean evaluateEntry(IndexInfo indexInfo, ExecutionContext context, Object keyVal) throws FunctionDomainException, TypeMismatchException, NameResolutionException, @@ -989,7 +919,7 @@ public class CompactRangeIndex extends AbstractIndex { return Boolean.FALSE; } } else { - return ((Boolean) result).booleanValue(); + return (Boolean) result; } } } @@ -1124,7 +1054,7 @@ public class CompactRangeIndex extends AbstractIndex { } public String toString() { - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); sb.append("No Keys = ").append(getNumberOfKeys()).append("\n"); sb.append("No Values = ").append(getNumberOfValues()).append("\n"); sb.append("No Uses = ").append(getTotalUses()).append("\n"); @@ -1134,43 +1064,39 @@ public class CompactRangeIndex extends AbstractIndex { } } - /** - * - */ class IMQEvaluator implements IndexedExpressionEvaluator { - private Cache cache; + private 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; /* - * Asif : The boolean if true indicates that the 0th iterator is on entries . If the 0th - * iterator is on collection of Region.Entry objects, then the RegionEntry object used in Index - * data objects is obtained directly from its corresponding Region.Entry object. However if the - * 0th iterator is not on entries then the boolean is false. In this case the additional - * projection attribute gives us the original value of the iterator while the Region.Entry - * object is obtained from 0th iterator. It is possible to have index being created on a Region - * Entry itself , instead of a Region. A Map operator( Compiled Index Operator) used with Region - * enables, us to create such indexes. In such case the 0th iterator, even if it represents a - * collection of Objects which are not Region.Entry objects, still the boolean remains true, as - * the Entry object can be easily obtained from the 0th iterator. In this case, the additional - * projection attribute s not null as it is used to evaluate the Entry object from the 0th - * iterator. + * The boolean if true indicates that the 0th iterator is on entries . If the 0th iterator is on + * collection of Region.Entry objects, then the RegionEntry object used in Index data objects is + * obtained directly from its corresponding Region.Entry object. However if the 0th iterator is + * not on entries then the boolean is false. In this case the additional projection attribute + * gives us the original value of the iterator while the Region.Entry object is obtained from + * 0th iterator. It is possible to have index being created on a Region Entry itself , instead + * of a Region. A Map operator( Compiled Index Operator) used with Region enables, us to create + * such indexes. In such case the 0th iterator, even if it represents a collection of Objects + * which are not Region.Entry objects, still the boolean remains true, as the Entry object can + * be easily obtained from the 0th iterator. In this case, the additional projection attribute s + * not null as it is used to evaluate the Entry object from the 0th iterator. */ private boolean isFirstItrOnEntry = false; - // Asif: List of modified iterators, not null only when the boolean + // List of modified iterators, not null only when the boolean // isFirstItrOnEntry is false. private List indexInitIterators = null; - // Asif : The additional Projection attribute representing the value of the + // The additional Projection attribute representing the value of the // original 0th iterator. If the isFirstItrOnEntry is false, then it is not // null. However if the isFirstItrOnEntry is true and this attribute is not // null, this indicates that the 0th iterator is derived using an individual // entry thru Map operator on the Region. private CompiledValue additionalProj = null; - // Asif : This is not null iff the boolean isFirstItrOnEntry is false. + // This is not null iff the boolean isFirstItrOnEntry is false. private CompiledValue modifiedIndexExpr = null; private ObjectType addnlProjType = null; private int initEntriesUpdated = 0; @@ -1187,7 +1113,7 @@ public class CompactRangeIndex extends AbstractIndex { this.canonicalIterNames = ((FunctionalIndexCreationHelper) helper).canonicalizedIteratorNames; this.rgn = helper.getRegion(); - // Asif : The modified iterators for optmizing Index cxreation + // The modified iterators for optmizing Index cxreation isFirstItrOnEntry = ((FunctionalIndexCreationHelper) helper).isFirstIteratorRegionEntry; additionalProj = ((FunctionalIndexCreationHelper) helper).additionalProj; Object params1[] = {new QRegion(rgn, false)}; @@ -1239,7 +1165,7 @@ public class CompactRangeIndex extends AbstractIndex { doNestedExpansion(1, expansionContext, expandedResults, lowerBoundKey, upperBoundKey, lowerBoundOperator, upperBoundOperator, value); } catch (Exception e) { - throw new IMQException(e) {}; + throw new IMQException(e); } } @@ -1268,16 +1194,8 @@ public class CompactRangeIndex extends AbstractIndex { } /** - * - * @param expansionContext - * @param expandedResults - * @param lowerBoundKey * @param upperBoundKey if null, we do not do an upperbound check (may need to change this if we * ever use null in a range query) - * @param lowerBoundOperator - * @param upperBoundOperator - * @param value - * @throws IMQException */ public void expand(ExecutionContext expansionContext, List expandedResults, Object lowerBoundKey, Object upperBoundKey, int lowerBoundOperator, int upperBoundOperator, @@ -1297,7 +1215,7 @@ public class CompactRangeIndex extends AbstractIndex { compResult = TypeUtils.compare(tupleIndexKey, upperBoundKey, upperBoundOperator); if (compResult instanceof Boolean) { Boolean ok = (Boolean) compResult; - if (!ok.booleanValue()) { + if (!ok) { return; } } @@ -1318,7 +1236,7 @@ public class CompactRangeIndex extends AbstractIndex { compResult = TypeUtils.compare(tupleIndexKey, lowerBoundKey, lowerBoundOperator); if (compResult instanceof Boolean) { Boolean ok = (Boolean) compResult; - if (!ok.booleanValue()) { + if (!ok) { return; } } @@ -1385,10 +1303,8 @@ public class CompactRangeIndex extends AbstractIndex { Support.Assert(this.indexResultSetType != null, "IMQEvaluator::evaluate:The StrcutType should have been initialized during index creation"); } catch (Exception e) { - e.printStackTrace(System.out); + logger.warn(e); throw new Error("Unable to reevaluate, this should not happen"); - } finally { - } return context; } @@ -1411,7 +1327,7 @@ public class CompactRangeIndex extends AbstractIndex { 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 + // 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); @@ -1450,13 +1366,12 @@ public class CompactRangeIndex extends AbstractIndex { } /** - * Asif : This function is used for creating Index data at the start - * + * This function is used for creating Index data at the start */ public void initializeIndex(boolean loadEntries) throws IMQException { this.initEntriesUpdated = 0; try { - // Asif: Since an index initialization can happen multiple times + // Since an index initialization can happen multiple times // for a given region, due to clear operation, we are using harcoded // scope ID of 1 , as otherwise if obtained from ExecutionContext // object, it will get incremented on very index initialization @@ -1512,15 +1427,15 @@ public class CompactRangeIndex extends AbstractIndex { } /* - * Asif : This function is used to obtain Index data at the time of index creation. Each element - * of the List is an Object Array of size 3. The 0th element of Object Array stores the value of - * Index Expression. The 1st element of ObjectArray contains the RegionEntry object ( If the - * booelan isFirstItrOnEntry is false, then the 0th iterator will give us the Region.Entry - * object which can be used to obtain the underlying RegionEntry object. If the boolean is true - * & additional projection attribute is not null, then the Region.Entry object can be obtained - * by evaluating the additional projection attribute. If the boolean isFirstItrOnEntry is tru e& - * additional projection attribute is null, then teh 0th iterator itself will evaluate to - * Region.Entry Object. + * This function is used to obtain Index data at the time of index creation. Each element of the + * List is an Object Array of size 3. The 0th element of Object Array stores the value of Index + * Expression. The 1st element of ObjectArray contains the RegionEntry object ( If the booelan + * isFirstItrOnEntry is false, then the 0th iterator will give us the Region.Entry object which + * can be used to obtain the underlying RegionEntry object. If the boolean is true & additional + * projection attribute is not null, then the Region.Entry object can be obtained by evaluating + * the additional projection attribute. If the boolean isFirstItrOnEntry is tru e& additional + * projection attribute is null, then teh 0th iterator itself will evaluate to Region.Entry + * Object. * * The 2nd element of Object Array contains the Struct object ( tuple) created. If the boolean * isFirstItrOnEntry is false, then the first attribute of the Struct object is obtained by @@ -1559,7 +1474,6 @@ public class CompactRangeIndex 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, @@ -1583,7 +1497,6 @@ public class CompactRangeIndex 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, @@ -1637,23 +1550,19 @@ public class CompactRangeIndex extends AbstractIndex { } } - // TODO :Asif: Test this function . // The struct type calculation is modified if the // 0th iterator is modified to make it dependent on Entry private ObjectType createIndexResultSetType() { List currentIterators = this.initContext.getCurrentIterators(); int len = currentIterators.size(); ObjectType type = null; - // String fieldNames[] = new String[len]; ObjectType fieldTypes[] = new ObjectType[len]; int start = this.isFirstItrOnEntry ? 0 : 1; for (; start < len; start++) { RuntimeIterator iter = (RuntimeIterator) currentIterators.get(start); - // fieldNames[start] = iter.getInternalId(); fieldTypes[start] = iter.getElementType(); } if (!this.isFirstItrOnEntry) { - // fieldNames[0] = "iter1"; fieldTypes[0] = addnlProjType; } type = (len == 1) ? fieldTypes[0] : new StructTypeImpl(this.canonicalIterNames, fieldTypes); http://git-wip-us.apache.org/repos/asf/geode/blob/57e93a5a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/DummyQRegion.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/DummyQRegion.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/DummyQRegion.java index 2b16686..3bee7d2 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/DummyQRegion.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/DummyQRegion.java @@ -12,12 +12,6 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ -/* - * DummyQRegion.java - * - * Created on March 15, 2005, 6:40 PM - */ - package org.apache.geode.cache.query.internal.index; import java.util.ArrayList; @@ -34,18 +28,14 @@ import org.apache.geode.cache.query.internal.ResultsSet; import org.apache.geode.cache.query.internal.types.TypeUtils; import org.apache.geode.cache.query.types.ObjectType; import org.apache.geode.internal.cache.CachedDeserializable; -import org.apache.geode.internal.cache.GemFireCacheImpl; +import org.apache.geode.internal.cache.HasCachePerfStats; import org.apache.geode.internal.cache.LocalRegion; import org.apache.geode.internal.cache.RegionEntry; -import org.apache.geode.internal.cache.RegionEntryContext; import org.apache.geode.internal.i18n.LocalizedStrings; import org.apache.geode.internal.offheap.StoredObject; import org.apache.geode.internal.offheap.annotations.Released; import org.apache.geode.internal.offheap.annotations.Retained; -/** - * - */ public class DummyQRegion extends QRegion { private RegionEntry entry = null; @@ -67,7 +57,7 @@ public class DummyQRegion extends QRegion { constraint = region.getAttributes().getKeyConstraint(); if (constraint != null) keyType = TypeUtils.getObjectType(constraint); - values = new ResultsBag(((GemFireCacheImpl) region.getCache()).getCachePerfStats()); + values = new ResultsBag(((HasCachePerfStats) region.getCache()).getCachePerfStats()); values.setElementType(valueType); keys = new ResultsSet(); keys.setElementType(keyType); @@ -107,7 +97,7 @@ public class DummyQRegion extends QRegion { @Override public Set keySet() { - return (ResultsSet) getKeys(); + return (Set) getKeys(); } @Override @@ -173,7 +163,7 @@ public class DummyQRegion extends QRegion { @Override public SelectResults getValues() { if (values == null) { - values = new ResultsBag(((GemFireCacheImpl) getRegion().getCache()).getCachePerfStats()); + values = new ResultsBag(((HasCachePerfStats) getRegion().getCache()).getCachePerfStats()); values.setElementType(valueType); } values.clear(); @@ -218,7 +208,7 @@ public class DummyQRegion extends QRegion { @Override public Set entries(boolean recursive) { - return (ResultsSet) getEntries(); + return (Set) getEntries(); } @Override http://git-wip-us.apache.org/repos/asf/geode/blob/57e93a5a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/FunctionalIndexCreationHelper.java ---------------------------------------------------------------------- diff --git a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/FunctionalIndexCreationHelper.java b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/FunctionalIndexCreationHelper.java index eb16207..4b61ae4 100644 --- a/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/FunctionalIndexCreationHelper.java +++ b/geode-core/src/main/java/org/apache/geode/cache/query/internal/index/FunctionalIndexCreationHelper.java @@ -12,17 +12,11 @@ * or implied. See the License for the specific language governing permissions and limitations under * the License. */ -/* - * IndexCreationHelper.java - * - * Created on March 16, 2005, 6:20 PM - */ package org.apache.geode.cache.query.internal.index; import java.util.ArrayList; import java.util.List; -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.IndexInvalidException; @@ -47,50 +41,62 @@ import org.apache.geode.cache.query.internal.QRegion; import org.apache.geode.cache.query.internal.RuntimeIterator; import org.apache.geode.cache.query.internal.parse.OQLLexerTokenTypes; import org.apache.geode.cache.query.types.ObjectType; +import org.apache.geode.internal.cache.InternalCache; import org.apache.geode.internal.cache.PartitionedRegion; import org.apache.geode.internal.i18n.LocalizedStrings; -/** - * - */ class FunctionalIndexCreationHelper extends IndexCreationHelper { + private boolean isMapTypeIndex; - // If true means pattern is *, if false & still map type index that means - // more than 1 specific keys + + /** + * If true means pattern is *, if false & still map type index that means more than 1 specific + * keys + */ private boolean isAllKeys = false; ExecutionContext context = null; - CompiledValue indexedExpr; - List fromClauseIterators; + + private CompiledValue indexedExpr; + + private List fromClauseIterators; + QRegion region; + String[] multiIndexKeysPattern; + Object[] mapKeys; + /** - * Asif : The Iterators for index creation are different then those which are used for index - * updates as in case of Index creation the 0th iterator is modified such that it always - * represents collection of Region.Entry objects. As a result all the rest of iterators as well as - * indexed expression have to be modified to appropriately resolve the dependency on 0th - * iterator.The missing link indicates the dependency. The original 0th iterator is evaluated as - * additional projection attribute. These changes provide significant improvement in Index - * creation as compared to previous method. In this approach the IMQ acts on all the entries of - * the region while in previous , it iterated over the individual entry of the Region & applied - * IMQ to it. + * The Iterators for index creation are different then those which are used for index updates as + * in case of Index creation the 0th iterator is modified such that it always represents + * collection of Region.Entry objects. As a result all the rest of iterators as well as indexed + * expression have to be modified to appropriately resolve the dependency on 0th iterator.The + * missing link indicates the dependency. The original 0th iterator is evaluated as additional + * projection attribute. These changes provide significant improvement in Index creation as + * compared to previous method. In this approach the IMQ acts on all the entries of the region + * while in previous , it iterated over the individual entry of the Region & applied IMQ to it. */ - List indexInitIterators = null; + CompiledValue missingLink = null; + CompiledValue additionalProj = null; + ObjectType addnlProjType = null; + CompiledValue modifiedIndexExpr = null; + boolean isFirstIteratorRegionEntry = false; + boolean isFirstIteratorRegionKey = false; - final String imports; - // TODO: Asif Remove the fromClause being passed as parameter to the - // constructor + private final String imports; + + // TODO: Remove the fromClause being passed as parameter to the constructor FunctionalIndexCreationHelper(String fromClause, String indexedExpression, - String projectionAttributes, String imports, Cache cache, ExecutionContext externalContext, - IndexManager imgr) throws IndexInvalidException { + String projectionAttributes, String imports, InternalCache cache, + ExecutionContext externalContext, IndexManager imgr) throws IndexInvalidException { super(fromClause, projectionAttributes, cache); if (externalContext == null) { this.context = new ExecutionContext(null, cache); @@ -102,27 +108,30 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { prepareFromClause(imgr); prepareIndexExpression(indexedExpression); prepareProjectionAttributes(projectionAttributes); - Object data[] = modfiyIterDefToSuiteIMQ((CompiledIteratorDef) fromClauseIterators.get(0)); + Object[] data = modiyyIterDefToSuiteIMQ((CompiledIteratorDef) this.fromClauseIterators.get(0)); if (data[0] == null || data[1] == null) { throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_INVALID_FROM_CLAUSE_0 .toLocalizedString(fromClause)); } - fromClauseIterators.remove(0); - fromClauseIterators.add(0, data[1]); - region = (QRegion) data[0]; + this.fromClauseIterators.remove(0); + this.fromClauseIterators.add(0, data[1]); + this.region = (QRegion) data[0]; } + @Override public List getIterators() { - return fromClauseIterators; + return this.fromClauseIterators; } + @Override public CompiledValue getCompiledIndexedExpression() { - return indexedExpr; + return this.indexedExpr; } + @Override public Region getRegion() { - return region.getRegion(); + return this.region.getRegion(); } @Override @@ -134,60 +143,56 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { return this.isAllKeys; } - - - /* - * Asif : The function is modified to optmize the index creation code. If the 0th iterator of from + /** + * The function is modified to optimize the index creation code. If the 0th iterator of from * clause is not on Entries, then the 0th iterator is replaced with that of entries & the value * corresponding to original iterator is derived from the 0th iterator as additional projection * attribute. All the other iterators & index expression if were dependent on 0th iterator are * also appropriately modified such that they are correctly derived on the modified 0th iterator. + *

+ * TODO: method is too complex for IDE to analyze -- refactor prepareFromClause */ private void prepareFromClause(IndexManager imgr) throws IndexInvalidException { - if (imports != null) { + if (this.imports != null) { this.compiler.compileImports(this.imports); } - List list = this.compiler.compileFromClause(fromClause); + List list = this.compiler.compileFromClause(this.fromClause); if (list == null) { throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_INVALID_FROM_CLAUSE_0 - .toLocalizedString(fromClause)); + .toLocalizedString(this.fromClause)); } - boolean isFromClauseNull = true; int size = list.size(); this.canonicalizedIteratorNames = new String[size]; this.canonicalizedIteratorDefinitions = new String[size]; - CompiledIteratorDef newItr = null; - StringBuffer tempBuff = new StringBuffer(); + StringBuilder tempBuff = new StringBuilder(); + boolean isFromClauseNull = true; + try { PartitionedRegion pr = this.context.getPartitionedRegion(); for (int i = 0; i < size; i++) { CompiledIteratorDef iterDef = (CompiledIteratorDef) list.get(i); iterDef.computeDependencies(this.context); RuntimeIterator rIter = iterDef.getRuntimeIterator(this.context); - context.addToIndependentRuntimeItrMapForIndexCreation(iterDef); + this.context.addToIndependentRuntimeItrMapForIndexCreation(iterDef); this.context.bindIterator(rIter); if (i != 0 && !iterDef.isDependentOnCurrentScope(this.context)) { throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_INVALID_FROM_CLAUSE_0_SUBSEQUENT_ITERATOR_EXPRESSIONS_IN_FROM_CLAUSE_MUST_BE_DEPENDENT_ON_PREVIOUS_ITERATORS - .toLocalizedString(fromClause)); + .toLocalizedString(this.fromClause)); } + String definition = rIter.getDefinition(); this.canonicalizedIteratorDefinitions[i] = definition; - // Asif: Bind the Index_Internal_ID to the RuntimeIterator + + // Bind the Index_Internal_ID to the RuntimeIterator this.canonicalizedIteratorNames[i] = imgr.putCanonicalizedIteratorNameIfAbsent(definition); if (pr != null) { - // if (iterDef.getCollectionExpr() instanceof CompiledRegion || - // iterDef.getCollectionExpr() instanceof CompiledPath) { - // pr.getIndexManager().putCanonicalizedIteratorName(pr.getFullPath(), - // this.canonicalizedIteratorNames[i]); - // } else { this.canonicalizedIteratorNames[i] = pr.getIndexManager().putCanonicalizedIteratorNameIfAbsent(definition); - // } } else { this.canonicalizedIteratorNames[i] = imgr.putCanonicalizedIteratorNameIfAbsent(definition); @@ -197,133 +202,132 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { tempBuff.append(definition).append(' ').append(this.canonicalizedIteratorNames[i]) .append(", "); isFromClauseNull = false; + CompiledIteratorDef newItr; + if (i == 0) { CompiledValue cv = iterDef.getCollectionExpr(); - addnlProjType = rIter.getElementType(); - String name = null; - if ((name = iterDef.getName()) == null || name.equals("")) { - // In case the name of iterator is null or balnk set it to - // index_internal_id + this.addnlProjType = rIter.getElementType(); + String name; + if ((name = iterDef.getName()) == null || name.isEmpty()) { + // In case the name of iterator is null or blank set it to index_internal_id name = this.canonicalizedIteratorNames[i]; } CompiledValue newCollExpr = new CompiledPath(new CompiledBindArgument(1), "entries"); - // TODO Asif : What if cv is not an instance of CompiledRegion + + // TODO: What if cv is not an instance of CompiledRegion if (cv instanceof CompiledRegion) { - missingLink = new CompiledPath(new CompiledID(name), "value"); - // missingLinkPath = name + ".value"; - additionalProj = missingLink; + this.missingLink = new CompiledPath(new CompiledID(name), "value"); + this.additionalProj = this.missingLink; + } else if (cv instanceof CompiledOperation || cv instanceof CompiledPath || cv instanceof CompiledIndexOperation) { - CompiledValue prevCV = null; + CompiledValue prevCV; List reconstruct = new ArrayList(); while (!(cv instanceof CompiledRegion)) { prevCV = cv; if (cv instanceof CompiledOperation) { reconstruct.add(0, ((CompiledOperation) cv).getArguments()); reconstruct.add(0, ((CompiledOperation) cv).getMethodName()); - cv = ((CompiledOperation) cv).getReceiver(context); + cv = ((CompiledOperation) cv).getReceiver(this.context); } else if (cv instanceof CompiledPath) { reconstruct.add(0, ((CompiledPath) cv).getTailID()); - cv = ((CompiledPath) cv).getReceiver(); + cv = cv.getReceiver(); } else if (cv instanceof CompiledIndexOperation) { reconstruct.add(0, ((CompiledIndexOperation) cv).getExpression()); - cv = ((CompiledIndexOperation) cv).getReceiver(); + cv = cv.getReceiver(); } else { throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_FUNCTIONALINDEXCREATIONHELPERPREPAREFROMCLAUSEFROM_CLAUSE_IS_NEITHER_A_COMPILEDPATH_NOR_COMPILEDOPERATION .toLocalizedString()); } - reconstruct.add(0, Integer.valueOf(prevCV.getType())); + reconstruct.add(0, prevCV.getType()); } - int firstTokenType = ((Integer) reconstruct.get(0)).intValue(); + + int firstTokenType = (Integer) reconstruct.get(0); if (firstTokenType == CompiledValue.PATH) { - // CompiledPath cp = (CompiledPath) reconstruct.get(1); String tailID = (String) reconstruct.get(1); + if (tailID.equals("asList") || tailID.equals("asSet") || tailID.equals("values") || tailID.equals("toArray") || tailID.equals("getValues")) { - missingLink = new CompiledPath(new CompiledID(name), "value"); - // missingLinkPath = name + ".value"; + this.missingLink = new CompiledPath(new CompiledID(name), "value"); } else if (tailID.equals("keys") || tailID.equals("getKeys") || tailID.equals("keySet")) { - missingLink = new CompiledPath(new CompiledID(name), "key"); - isFirstIteratorRegionKey = true; - // missingLinkPath = name + ".key"; + this.missingLink = new CompiledPath(new CompiledID(name), "key"); + this.isFirstIteratorRegionKey = true; } else if (tailID.equals("entries") || tailID.equals("getEntries") || tailID.equals("entrySet")) { - isFirstIteratorRegionEntry = true; + this.isFirstIteratorRegionEntry = true; } else { throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_FUNCTIONALINDEXCREATIONHELPERPREPAREFROMCLAUSEFROM_CLAUSE_DOES_NOT_EVALUATE_TO_VALID_COLLECTION .toLocalizedString()); } + remove(reconstruct, 2, 0); - int secondTokenType = - (reconstruct.size() > 1) ? ((Integer) reconstruct.get(0)).intValue() : -1; - if (!isFirstIteratorRegionEntry - && (secondTokenType == OQLLexerTokenTypes.TOK_LBRACK)) { - // Asif: If the field just next to region , is values or - // getValues & next to it is - // CompiledIndexOpn, it indirectly means Map operation & we are - // able to take care of it by adding a flag in CompiledIndexOp - // which - // indicates to it whether to return entry or value.But if the - // field - // is asList or toArray , we have a problem as we don't have a - // corresponding - // list of entries. If the field is keys , an exception should - // be thrown - // as IndexOpn on set is not defined. + int secondTokenType = reconstruct.size() > 1 ? (Integer) reconstruct.get(0) : -1; + if (!this.isFirstIteratorRegionEntry + && secondTokenType == OQLLexerTokenTypes.TOK_LBRACK) { + + // If the field just next to region , is values or getValues & next to it is + // CompiledIndexOpn, it indirectly means Map operation & we are able to take care of + // it by adding a flag in CompiledIndexOp which indicates to it whether to return + // entry or value. But if the field is asList or toArray , we have a problem as we + // don't have a corresponding list of entries. If the field is keys , an exception + // should be thrown as IndexOpn on set is not defined. if (tailID.equals("values") || tailID.equals("getValues")) { boolean returnEntryForRegionCollection = true; - additionalProj = new CompiledIndexOperation(new CompiledBindArgument(1), + this.additionalProj = new CompiledIndexOperation(new CompiledBindArgument(1), (CompiledValue) reconstruct.get(1), returnEntryForRegionCollection); this.isFirstIteratorRegionEntry = true; + } else if (tailID.equals("toList") || tailID.equals("toArray")) { - // TODO:Asif . This needs to be supported + // TODO: add support for toList and toArray throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_FUNCTIONALINDEXCREATIONHELPERPREPAREFROMCLAUSETOLIST_TOARRAY_NOT_SUPPORTED .toLocalizedString()); + } else { throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_FUNCTIONALINDEXCREATIONHELPERPREPAREFROMCLAUSETOLIST_TOARRAY_NOT_SUPPORTED .toLocalizedString()); } remove(reconstruct, 2, 0); - } else if (!isFirstIteratorRegionEntry + + } else if (!this.isFirstIteratorRegionEntry && (secondTokenType == OQLLexerTokenTypes.METHOD_INV || secondTokenType == CompiledValue.PATH) && (tailID.equals("values") || tailID.equals("getValues") || tailID.equals("keySet") || tailID.equals("keys") || tailID.equals("getKeys"))) { - // Asif :Check if the second token name is toList or toArray or - // asSet.We need to remove those + + // Check if the second token name is toList or toArray or asSet.We need to remove + // those String secTokName = (String) reconstruct.get(1); if (secTokName.equals("asList") || secTokName.equals("asSet") || secTokName.equals("toArray")) { - remove(reconstruct, ((secondTokenType == OQLLexerTokenTypes.METHOD_INV) ? 3 : 2), - 0); + remove(reconstruct, secondTokenType == OQLLexerTokenTypes.METHOD_INV ? 3 : 2, 0); } } + } else if (firstTokenType == OQLLexerTokenTypes.TOK_LBRACK) { boolean returnEntryForRegionCollection = true; - additionalProj = new CompiledIndexOperation(new CompiledBindArgument(1), + this.additionalProj = new CompiledIndexOperation(new CompiledBindArgument(1), (CompiledValue) reconstruct.get(1), returnEntryForRegionCollection); this.isFirstIteratorRegionEntry = true; + } else if (firstTokenType == OQLLexerTokenTypes.METHOD_INV) { String methodName = (String) reconstruct.get(1); if (methodName.equals("asList") || methodName.equals("asSet") || methodName.equals("values") || methodName.equals("toArray") || methodName.equals("getValues")) { - missingLink = new CompiledPath(new CompiledID(name), "value"); - // missingLinkPath = name + ".value"; + this.missingLink = new CompiledPath(new CompiledID(name), "value"); } else if (methodName.equals("keys") || methodName.equals("getKeys") || methodName.equals("keySet")) { - missingLink = new CompiledPath(new CompiledID(name), "key"); - isFirstIteratorRegionKey = true; - // missingLinkPath = name + ".key"; + this.missingLink = new CompiledPath(new CompiledID(name), "key"); + this.isFirstIteratorRegionKey = true; } else if (methodName.equals("entries") || methodName.equals("getEntries") || methodName.equals("entrySet")) { - isFirstIteratorRegionEntry = true; + this.isFirstIteratorRegionEntry = true; List args = (List) reconstruct.get(2); if (args != null && args.size() == 1) { Object obj = args.get(0); @@ -334,17 +338,18 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { } } } + remove(reconstruct, 3, 0); - int secondTokenType = - (reconstruct.size() > 1) ? ((Integer) reconstruct.get(0)).intValue() : -1; - if (!isFirstIteratorRegionEntry - && (secondTokenType == OQLLexerTokenTypes.TOK_LBRACK)) { + int secondTokenType = reconstruct.size() > 1 ? (Integer) reconstruct.get(0) : -1; + if (!this.isFirstIteratorRegionEntry + && secondTokenType == OQLLexerTokenTypes.TOK_LBRACK) { + if (methodName.equals("values") || methodName.equals("getValues")) { boolean returnEntryForRegionCollection = true; newCollExpr = new CompiledIndexOperation(new CompiledBindArgument(1), (CompiledValue) reconstruct.get(1), returnEntryForRegionCollection); } else if (methodName.equals("toList") || methodName.equals("toArray")) { - // TODO:Asif . This needs to be supported + // TODO: add support for toList and toArray throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_FUNCTIONALINDEXCREATIONHELPERPREPAREFROMCLAUSETOLIST_TOARRAY_NOT_SUPPORTED_YET .toLocalizedString()); @@ -353,38 +358,40 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { LocalizedStrings.FunctionalIndexCreationHelper_FUNCTIONALINDEXCREATIONHELPERPREPAREFROMCLAUSETOLIST_TOARRAY_NOT_SUPPORTED_YET .toLocalizedString()); } + remove(reconstruct, 2, 0); - } else if (!isFirstIteratorRegionEntry + } else if (!this.isFirstIteratorRegionEntry && (secondTokenType == OQLLexerTokenTypes.METHOD_INV || secondTokenType == CompiledValue.PATH) && (methodName.equals("values") || methodName.equals("getValues") || methodName.equals("keys") || methodName.equals("getKeys") || methodName.equals("keySet"))) { - // Asif :Check if the second token name is toList or toArray or - // asSet.We need to remove those + + // Check if the second token name is toList or toArray or asSet.We need to remove + // those String secTokName = (String) reconstruct.get(1); if (secTokName.equals("asList") || secTokName.equals("asSet") || secTokName.equals("toArray")) { - remove(reconstruct, ((secondTokenType == OQLLexerTokenTypes.METHOD_INV) ? 3 : 2), - 0); + remove(reconstruct, secondTokenType == OQLLexerTokenTypes.METHOD_INV ? 3 : 2, 0); } } } - if (!isFirstIteratorRegionEntry) { - additionalProj = missingLink; + + if (!this.isFirstIteratorRegionEntry) { + this.additionalProj = this.missingLink; int len = reconstruct.size(); for (int j = 0; j < len; ++j) { Object obj = reconstruct.get(j); if (obj instanceof Integer) { - int tokenType = ((Integer) obj).intValue(); + int tokenType = (Integer) obj; if (tokenType == CompiledValue.PATH) { - additionalProj = - new CompiledPath(additionalProj, (String) reconstruct.get(++j)); + this.additionalProj = + new CompiledPath(this.additionalProj, (String) reconstruct.get(++j)); } else if (tokenType == OQLLexerTokenTypes.TOK_LBRACK) { - additionalProj = new CompiledIndexOperation(additionalProj, + this.additionalProj = new CompiledIndexOperation(this.additionalProj, (CompiledValue) reconstruct.get(++j)); } else if (tokenType == OQLLexerTokenTypes.METHOD_INV) { - additionalProj = new CompiledOperation(additionalProj, + this.additionalProj = new CompiledOperation(this.additionalProj, (String) reconstruct.get(++j), (List) reconstruct.get(++j)); } } @@ -395,41 +402,44 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { LocalizedStrings.FunctionalIndexCreationHelper_FUNCTIONALINDEXCREATIONHELPERPREPAREFROMCLAUSEFROM_CLAUSE_IS_NEITHER_A_COMPILEDPATH_NOR_COMPILEDOPERATION .toLocalizedString()); } + if (!this.isFirstIteratorRegionEntry) { newItr = new CompiledIteratorDef(name, null, newCollExpr); this.indexInitIterators = new ArrayList(); - indexInitIterators.add(newItr); + this.indexInitIterators.add(newItr); } + } else if (!this.isFirstIteratorRegionEntry) { newItr = iterDef; - if (rIter.getDefinition().indexOf(this.canonicalizedIteratorNames[0]) != -1) { - newItr = - (CompiledIteratorDef) getModifiedDependentCompiledValue(context, i, iterDef, true); + if (rIter.getDefinition().contains(this.canonicalizedIteratorNames[0])) { + newItr = (CompiledIteratorDef) getModifiedDependentCompiledValue(this.context, i, + iterDef, true); } this.indexInitIterators.add(newItr); } } + } catch (IndexInvalidException e) { + throw e; } catch (Exception e) { - if (e instanceof IndexInvalidException) - throw (IndexInvalidException) e; throw new IndexInvalidException(e); } if (isFromClauseNull) throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_INVALID_FROM_CLAUSE_0 - .toLocalizedString(fromClause)); + .toLocalizedString(this.fromClause)); this.fromClause = tempBuff.substring(0, tempBuff.length() - 2); this.fromClauseIterators = list; } - /* - * Asif: This fuinction is modified so that if the indexed expression has any dependency on the - * 0th iterator, then it needs to modified by using the missing link so that it is derivable from - * the 0th iterator. + /** + * This function is modified so that if the indexed expression has any dependency on the 0th + * iterator, then it needs to modified by using the missing link so that it is derivable from the + * 0th iterator. + *

+ * TODO: refactor large method prepareIndexExpression */ private void prepareIndexExpression(String indexedExpression) throws IndexInvalidException { CompiledValue expr = this.compiler.compileQuery(indexedExpression); - // List indexedExprs = this.compiler.compileProjectionAttributes(indexedExpression); if (expr == null) { throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_INVALID_INDEXED_EXPRESSION_0 @@ -438,73 +448,73 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { if (expr instanceof CompiledUndefined || expr instanceof CompiledLiteral || expr instanceof CompiledComparison || expr instanceof CompiledBindArgument - || expr instanceof CompiledNegation) + || expr instanceof CompiledNegation) { throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_INVALID_INDEXED_EXPRESSION_0 .toLocalizedString(indexedExpression)); + } + try { - StringBuffer sb = new StringBuffer(); + StringBuilder sb = new StringBuilder(); if (expr instanceof MapIndexable) { MapIndexable mi = (MapIndexable) expr; - // CompiledIndexOperation cio = (CompiledIndexOperation)expr; List indexingKeys = mi.getIndexingKeys(); + if (indexingKeys.size() == 1 && indexingKeys.get(0) == CompiledValue.MAP_INDEX_ALL_KEYS) { this.isMapTypeIndex = true; this.isAllKeys = true; // Strip the index operator expr = mi.getRecieverSansIndexArgs(); - expr.generateCanonicalizedExpression(sb, context); + expr.generateCanonicalizedExpression(sb, this.context); sb.append('[').append('*').append(']'); } else if (indexingKeys.size() == 1) { - expr.generateCanonicalizedExpression(sb, context); + expr.generateCanonicalizedExpression(sb, this.context); + } else { this.isMapTypeIndex = true; this.multiIndexKeysPattern = new String[indexingKeys.size()]; this.mapKeys = new Object[indexingKeys.size()]; expr = mi.getRecieverSansIndexArgs(); - expr.generateCanonicalizedExpression(sb, context); + expr.generateCanonicalizedExpression(sb, this.context); sb.append('['); String prefixStr = sb.toString(); - StringBuffer buff2 = new StringBuffer(); + StringBuilder sb2 = new StringBuilder(); int size = indexingKeys.size(); for (int j = 0; j < size; ++j) { CompiledValue cv = indexingKeys.get(size - j - 1); - this.mapKeys[size - j - 1] = cv.evaluate(context); - StringBuffer sbuff = new StringBuffer(); - cv.generateCanonicalizedExpression(sbuff, context); - sbuff.insert(0, prefixStr); - sbuff.append(']'); - this.multiIndexKeysPattern[j] = sbuff.toString(); - cv.generateCanonicalizedExpression(buff2, context); - buff2.insert(0, ','); + this.mapKeys[size - j - 1] = cv.evaluate(this.context); + StringBuilder sb3 = new StringBuilder(); + cv.generateCanonicalizedExpression(sb3, this.context); + sb3.insert(0, prefixStr); + sb3.append(']'); + this.multiIndexKeysPattern[j] = sb3.toString(); + cv.generateCanonicalizedExpression(sb2, this.context); + sb2.insert(0, ','); } - buff2.deleteCharAt(0); - sb.append(buff2.toString()); + sb2.deleteCharAt(0); + sb.append(sb2); sb.append(']'); } } else { - expr.generateCanonicalizedExpression(sb, context); + expr.generateCanonicalizedExpression(sb, this.context); } - // expr.generateCanonicalizedExpression(sb, this.context); this.indexedExpression = sb.toString(); - // String tempStr = this.indexedExpression; - modifiedIndexExpr = expr; + this.modifiedIndexExpr = expr; if (!this.isFirstIteratorRegionEntry - && this.indexedExpression.indexOf(this.canonicalizedIteratorNames[0]) >= 0) { - modifiedIndexExpr = getModifiedDependentCompiledValue(context, -1, expr, true); + && this.indexedExpression.contains(this.canonicalizedIteratorNames[0])) { + this.modifiedIndexExpr = getModifiedDependentCompiledValue(this.context, -1, expr, true); } } catch (Exception e) { - // e.printStackTrace(); throw new IndexInvalidException( LocalizedStrings.FunctionalIndexCreationHelper_INVALID_INDEXED_EXPRESSION_0 .toLocalizedString(indexedExpression), e); } - indexedExpr = expr; + this.indexedExpr = expr; } private void prepareProjectionAttributes(String projectionAttributes) @@ -517,30 +527,30 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { this.projectionAttributes = projectionAttributes; } - private Object[] modfiyIterDefToSuiteIMQ(CompiledIteratorDef iterDef) + private Object[] modiyyIterDefToSuiteIMQ(CompiledIteratorDef iterDef) throws IndexInvalidException { - Object retValues[] = {null, null}; + Object[] retValues = {null, null}; try { CompiledValue def = iterDef.getCollectionExpr(); - // System.out.println("def = "+def); if (def instanceof CompiledRegion) { CompiledBindArgument bindArg = new CompiledBindArgument(1); CompiledIteratorDef newDef = new CompiledIteratorDef(iterDef.getName(), null, bindArg); - retValues[0] = def.evaluate(context); + retValues[0] = def.evaluate(this.context); retValues[1] = newDef; return retValues; } + if (def instanceof CompiledPath || def instanceof CompiledOperation || def instanceof CompiledIndexOperation) { CompiledValue cv = def; - CompiledValue prevCV = null; List reconstruct = new ArrayList(); + while (!(cv instanceof CompiledRegion)) { - prevCV = cv; + CompiledValue prevCV = cv; if (cv instanceof CompiledOperation) { reconstruct.add(0, ((CompiledOperation) cv).getArguments()); reconstruct.add(0, ((CompiledOperation) cv).getMethodName()); - cv = ((CompiledOperation) cv).getReceiver(context); + cv = ((CompiledOperation) cv).getReceiver(this.context); } else if (cv instanceof CompiledPath) { reconstruct.add(0, ((CompiledPath) cv).getTailID()); cv = ((CompiledPath) cv).getReceiver(); @@ -552,15 +562,16 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { LocalizedStrings.FunctionalIndexCreationHelper_FUNCTIONALINDEXCREATIONHELPERPREPAREFROMCLAUSEFROM_CLAUSE_IS_NEITHER_A_COMPILEDPATH_NOR_COMPILEDOPERATION .toLocalizedString()); } - reconstruct.add(0, Integer.valueOf(prevCV.getType())); + reconstruct.add(0, prevCV.getType()); } + CompiledValue v = cv; cv = new CompiledBindArgument(1); int len = reconstruct.size(); for (int j = 0; j < len; ++j) { Object obj = reconstruct.get(j); if (obj instanceof Integer) { - int tokenType = ((Integer) obj).intValue(); + int tokenType = (Integer) obj; if (tokenType == CompiledValue.PATH) { cv = new CompiledPath(cv, (String) reconstruct.get(++j)); } else if (tokenType == OQLLexerTokenTypes.TOK_LBRACK) { @@ -571,8 +582,9 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { } } } + CompiledIteratorDef newDef = new CompiledIteratorDef(iterDef.getName(), null, cv); - retValues[0] = v.evaluate(context); + retValues[0] = v.evaluate(this.context); retValues[1] = newDef; return retValues; } @@ -582,46 +594,49 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { return retValues; } - /* - * Asif : This function is used to correct the complied value's dependency , in case the - * compiledvalue is dependent on the 0th RuntimeIterator in some way. Thus the dependent compiled - * value is prefixed with the missing link so that it is derivable from the 0th iterator. + /** + * This function is used to correct the complied value's dependency , in case the compiledValue is + * dependent on the 0th RuntimeIterator in some way. Thus the dependent compiled value is prefixed + * with the missing link so that it is derivable from the 0th iterator. */ private CompiledValue getModifiedDependentCompiledValue(ExecutionContext context, int currItrID, CompiledValue cv, boolean isDependent) throws AmbiguousNameException, TypeMismatchException, NameResolutionException { + if (cv instanceof CompiledIteratorDef) { CompiledIteratorDef iterDef = (CompiledIteratorDef) cv; RuntimeIterator rItr = (RuntimeIterator) context.getCurrentIterators().get(currItrID); String canonFrmClause = rItr.getDefinition(); - if (canonFrmClause.startsWith(this.canonicalizedIteratorNames[0])) - isDependent = true; - else - isDependent = false; + + // TODO: original value of isDependent is always ignored + isDependent = canonFrmClause.startsWith(this.canonicalizedIteratorNames[0]); + return new CompiledIteratorDef(iterDef.getName(), rItr.getElementType(), getModifiedDependentCompiledValue(context, currItrID, iterDef.getCollectionExpr(), isDependent)); + } else if (cv instanceof CompiledPath) { CompiledPath path = (CompiledPath) cv; return new CompiledPath( getModifiedDependentCompiledValue(context, currItrID, path.getReceiver(), isDependent), path.getTailID()); + } else if (cv instanceof CompiledOperation) { CompiledOperation oper = (CompiledOperation) cv; List list = oper.getArguments(); - int len = list.size(); List newList = new ArrayList(); - for (int i = 0; i < len; ++i) { - CompiledValue cv1 = (CompiledValue) list.get(i); - StringBuffer sbuff = new StringBuffer(); - cv1.generateCanonicalizedExpression(sbuff, context); - if (sbuff.toString().startsWith(this.canonicalizedIteratorNames[0])) { + for (Object aList : list) { + CompiledValue cv1 = (CompiledValue) aList; + StringBuilder sb = new StringBuilder(); + cv1.generateCanonicalizedExpression(sb, context); + if (sb.toString().startsWith(this.canonicalizedIteratorNames[0])) { newList.add(getModifiedDependentCompiledValue(context, currItrID, cv1, true)); } else { newList.add(getModifiedDependentCompiledValue(context, currItrID, cv1, false)); } } - // Asif: What if the receiver is null? + + // What if the receiver is null? CompiledValue rec = oper.getReceiver(context); if (rec == null) { if (isDependent) { @@ -633,6 +648,7 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { getModifiedDependentCompiledValue(context, currItrID, rec, isDependent), oper.getMethodName(), newList); } + } else if (cv instanceof CompiledFunction) { CompiledFunction cf = (CompiledFunction) cv; CompiledValue[] cvArray = cf.getArguments(); @@ -641,37 +657,38 @@ class FunctionalIndexCreationHelper extends IndexCreationHelper { CompiledValue[] newCvArray = new CompiledValue[len]; for (int i = 0; i < len; ++i) { CompiledValue cv1 = cvArray[i]; - StringBuffer sbuff = new StringBuffer(); - cv1.generateCanonicalizedExpression(sbuff, context); - if (sbuff.toString().startsWith(this.canonicalizedIteratorNames[0])) { + StringBuilder sb = new StringBuilder(); + cv1.generateCanonicalizedExpression(sb, context); + if (sb.toString().startsWith(this.canonicalizedIteratorNames[0])) { newCvArray[i] = getModifiedDependentCompiledValue(context, currItrID, cv1, true); } else { newCvArray[i] = getModifiedDependentCompiledValue(context, currItrID, cv1, false); } } return new CompiledFunction(newCvArray, function); + } else if (cv instanceof CompiledID) { CompiledID id = (CompiledID) cv; RuntimeIterator rItr0 = (RuntimeIterator) context.getCurrentIterators().get(0); if (isDependent) { - String name = null; + String name; if ((name = rItr0.getName()) != null && name.equals(id.getId())) { - // Asif: The CompiledID is a RuneTimeIterator & so it needs to be - // replaced by the missing link + // The CompiledID is a RuneTimeIterator & so it needs to be replaced by the missing link return this.missingLink; } else { - // Asif: The compiledID is a compiledpath - return new CompiledPath(missingLink, id.getId()); + // The compiledID is a compiledPath + return new CompiledPath(this.missingLink, id.getId()); } } else { return cv; } + } else if (cv instanceof CompiledIndexOperation) { CompiledIndexOperation co = (CompiledIndexOperation) cv; CompiledValue cv1 = co.getExpression(); - StringBuffer sbuff = new StringBuffer(); - cv1.generateCanonicalizedExpression(sbuff, context); - if (sbuff.toString().startsWith(this.canonicalizedIteratorNames[0])) { + StringBuilder sb = new StringBuilder(); + cv1.generateCanonicalizedExpression(sb, context); + if (sb.toString().startsWith(this.canonicalizedIteratorNames[0])) { cv1 = getModifiedDependentCompiledValue(context, currItrID, cv1, true); } else { cv1 = getModifiedDependentCompiledValue(context, currItrID, cv1, false);