You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org> on 2016/08/24 16:36:29 UTC

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Murtadha Hubail has uploaded a new change for review.

  https://asterix-gerrit.ics.uci.edu/1109

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................

ASTERIXDB-1451: Fix type propagation to enforced indexes

This change includes the following:
- Introduce cast function in case of delete operation
  after the primary index to ensure types are passed
  correctly to enforced indexes.
- Introduce cast function in case of upsert operation
  before old secondary keys extraction to ensure types
  are passed correctly to enforced indexes.
- Sonar-Qube fixes.

Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
14 files changed, 415 insertions(+), 82 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/09/1109/1

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
index c64258f..c1c983e 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
@@ -20,7 +20,6 @@
 
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Stack;
@@ -136,7 +135,7 @@
              * inserted/deleted.
              */
             while (fid != AsterixBuiltinFunctions.OPEN_RECORD_CONSTRUCTOR) {
-                if (op2.getInputs().size() == 0) {
+                if (op2.getInputs().isEmpty()) {
                     return false;
                 }
                 op2 = (AbstractLogicalOperator) op2.getInputs().get(0).getValue();
@@ -146,8 +145,8 @@
                 AssignOperator assignOp = (AssignOperator) op2;
                 ILogicalExpression assignExpr = assignOp.getExpressions().get(0).getValue();
                 if (assignExpr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                    ScalarFunctionCallExpression funcExpr =
-                            (ScalarFunctionCallExpression) assignOp.getExpressions().get(0).getValue();
+                    ScalarFunctionCallExpression funcExpr = (ScalarFunctionCallExpression) assignOp.getExpressions()
+                            .get(0).getValue();
                     fid = funcExpr.getFunctionIdentifier();
                 }
             }
@@ -187,9 +186,6 @@
         if (dataset.hasMetaPart()) {
             metaType = (ARecordType) mp.findType(dataset.getMetaItemTypeDataverseName(), dataset.getMetaItemTypeName());
         }
-        // recType might be replaced with enforced record type and we want to keep a reference to the original record
-        // type
-        ARecordType originalRecType = recType;
         List<Index> indexes = mp.getDatasetIndexes(dataset.getDataverseName(), dataset.getDatasetName());
         // Set the top operator pointer to the primary IndexInsertDeleteOperator
         ILogicalOperator currentTop = op1;
@@ -197,13 +193,7 @@
 
         // Put an n-gram or a keyword index in the later stage of index-update,
         // since TokenizeOperator needs to be involved.
-        Collections.sort(indexes, new Comparator<Index>() {
-            @Override
-            public int compare(Index o1, Index o2) {
-                return o1.getIndexType().ordinal() - o2.getIndexType().ordinal();
-            }
-
-        });
+        Collections.sort(indexes, (o1, o2) -> o1.getIndexType().ordinal() - o2.getIndexType().ordinal());
 
         // Check whether multiple indexes exist
         int secondaryIndexTotalCnt = 0;
@@ -222,13 +212,13 @@
         // Prepare filtering field information (This is the filter created using the "filter with" key word in the
         // create dataset ddl)
         List<String> filteringFields = ((InternalDatasetDetails) dataset.getDatasetDetails()).getFilterField();
-        List<LogicalVariable> filteringVars = null;
+        List<LogicalVariable> filteringVars;
         List<Mutable<ILogicalExpression>> filteringExpressions = null;
 
         if (filteringFields != null) {
             // The filter field var already exists. we can simply get it from the insert op
-            filteringVars = new ArrayList<LogicalVariable>();
-            filteringExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+            filteringVars = new ArrayList<>();
+            filteringExpressions = new ArrayList<>();
             for (Mutable<ILogicalExpression> filteringExpression : insertOp.getAdditionalFilteringExpressions()) {
                 filteringExpression.getValue().getUsedVariables(filteringVars);
                 for (LogicalVariable var : filteringVars) {
@@ -238,12 +228,14 @@
             }
         }
         LogicalVariable enforcedRecordVar = recordVar;
+        LogicalVariable upsertEnforcedRecordVar = insertOp.getPrevRecordVar();
 
         /*
          * if the index is enforcing field types (For open indexes), We add a cast
          * operator to ensure type safety
          */
-        if (insertOp.getOperation() == Kind.INSERT || insertOp.getOperation() == Kind.UPSERT) {
+        if (insertOp.getOperation() == Kind.INSERT || insertOp.getOperation() == Kind.UPSERT
+                || insertOp.getOperation() == Kind.DELETE) {
             try {
                 DatasetDataSource ds = (DatasetDataSource) (insertOp.getDataSource());
                 ARecordType insertRecType = (ARecordType) ds.getItemType();
@@ -261,18 +253,42 @@
                             .add(new MutableObject<ILogicalExpression>(insertOp.getPayloadExpression().getValue()));
                     TypeCastUtils.setRequiredAndInputTypes(castFunc, enforcedType, insertRecType);
                     // AssignOperator puts in the cast var the casted record
-                    AssignOperator castedRecordAssignOperator =
-                            new AssignOperator(castedRecVar, new MutableObject<ILogicalExpression>(castFunc));
-                    // Connect the current top of the plan to the cast operator
-                    castedRecordAssignOperator.getInputs().addAll(currentTop.getInputs());
-                    currentTop.getInputs().clear();
-                    currentTop.getInputs().add(new MutableObject<>(castedRecordAssignOperator));
+                    AssignOperator castedRecordAssignOperator = new AssignOperator(castedRecVar,
+                            new MutableObject<ILogicalExpression>(castFunc));
+                    if (insertOp.getOperation() == Kind.INSERT) {
+                        // Connect the current top of the plan to the cast operator
+                        connectCastOperator(context, currentTop, castedRecordAssignOperator);
+                    } else if (insertOp.getOperation() == Kind.DELETE) {
+                        /*
+                         * in the delete case, the primary index might store records with
+                         * a different type than an enforced index. Therefore, we introduce
+                         * a cast after the primary index to ensure the correct enforced type 
+                         * is passed to the enforced indexes.
+                         */
+                        castedRecordAssignOperator.getInputs().add(new MutableObject<>(currentTop));
+                        currentTop = castedRecordAssignOperator;
+                        context.computeAndSetTypeEnvironmentForOperator(currentTop);
+                    } else if (insertOp.getOperation() == Kind.UPSERT) {
+                        /*
+                         * in case of upsert, we introduce a cast before extracting the
+                         * old secondary keys and passing them to enforced indexes
+                         */
+                        upsertEnforcedRecordVar = context.newVar();
+                        AbstractFunctionCallExpression upsertSecondaryKeysCastFunc = new ScalarFunctionCallExpression(
+                                FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.CAST_TYPE));
+                        upsertSecondaryKeysCastFunc.getArguments()
+                                .add(new MutableObject<>(new VariableReferenceExpression(insertOp.getPrevRecordVar())));
+                        TypeCastUtils.setRequiredAndInputTypes(upsertSecondaryKeysCastFunc, enforcedType,
+                                insertRecType);
+                        AssignOperator upsertCastedRecordAssignOperator = new AssignOperator(upsertEnforcedRecordVar,
+                                new MutableObject<ILogicalExpression>(upsertSecondaryKeysCastFunc));
+                        upsertCastedRecordAssignOperator.getInputs().add(new MutableObject<>(currentTop));
+                        context.computeAndSetTypeEnvironmentForOperator(upsertCastedRecordAssignOperator);
+                        connectCastOperator(context, currentTop, castedRecordAssignOperator);
+                        currentTop = upsertCastedRecordAssignOperator;
+                    }
                     enforcedRecordVar = castedRecVar;
                     recType = enforcedType;
-                    context.computeAndSetTypeEnvironmentForOperator(castedRecordAssignOperator);
-                    context.computeAndSetTypeEnvironmentForOperator(currentTop);
-                    // We don't need to cast the old rec, we just need an assignment function that extracts the SK
-                    // and an expression which reference the new variables.
                 }
             } catch (AsterixException e) {
                 throw new AlgebricksException(e);
@@ -301,10 +317,10 @@
             // Get the secondary fields names and types
             List<List<String>> secondaryKeyFields = index.getKeyFieldNames();
             List<IAType> secondaryKeyTypes = index.getKeyFieldTypes();
-            List<LogicalVariable> secondaryKeyVars = new ArrayList<LogicalVariable>();
+            List<LogicalVariable> secondaryKeyVars = new ArrayList<>();
             List<Integer> indicators = index.getKeyFieldSourceIndicators();
-            List<Mutable<ILogicalExpression>> expressions = new ArrayList<Mutable<ILogicalExpression>>();
-            List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+            List<Mutable<ILogicalExpression>> expressions = new ArrayList<>();
+            List<Mutable<ILogicalExpression>> secondaryExpressions = new ArrayList<>();
 
             for (int i = 0; i < secondaryKeyFields.size(); i++) {
                 List<String> secondaryKey = secondaryKeyFields.get(i);
@@ -320,19 +336,18 @@
             // Used with upsert operation
             // in case of upsert, we need vars and expressions for the old SK as well.
             List<LogicalVariable> prevSecondaryKeyVars = null;
-            List<Mutable<ILogicalExpression>> prevExpressions = null;
+            List<Mutable<ILogicalExpression>> prevExpressions;
             List<Mutable<ILogicalExpression>> prevSecondaryExpressions = null;
             AssignOperator prevSecondaryKeyAssign = null;
             if (insertOp.getOperation() == Kind.UPSERT) {
-                prevSecondaryKeyVars = new ArrayList<LogicalVariable>();
-                prevExpressions = new ArrayList<Mutable<ILogicalExpression>>();
-                prevSecondaryExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+                prevSecondaryKeyVars = new ArrayList<>();
+                prevExpressions = new ArrayList<>();
+                prevSecondaryExpressions = new ArrayList<>();
                 for (int i = 0; i < secondaryKeyFields.size(); i++) {
                     List<String> secondaryKey = secondaryKeyFields.get(i);
                     prepareVarAndExpression(secondaryKey,
-                            (indicators.get(i).intValue() == 0) ? originalRecType.getFieldNames()
-                                    : metaType.getFieldNames(),
-                            (indicators.get(i).intValue() == 0) ? insertOp.getPrevRecordVar()
+                            (indicators.get(i).intValue() == 0) ? recType.getFieldNames() : metaType.getFieldNames(),
+                            (indicators.get(i).intValue() == 0) ? upsertEnforcedRecordVar
                                     : insertOp.getPrevAdditionalNonFilteringVars().get(0),
                             prevExpressions, prevSecondaryKeyVars, context);
                 }
@@ -407,11 +422,11 @@
 
                     // Check the field type of the secondary key.
                     IAType secondaryKeyType = null;
-                    Pair<IAType, Boolean> keyPairType =
-                            Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0), recType);
+                    Pair<IAType, Boolean> keyPairType = Index.getNonNullableKeyFieldType(secondaryKeyFields.get(0),
+                            recType);
                     secondaryKeyType = keyPairType.first;
 
-                    List<Object> varTypes = new ArrayList<Object>();
+                    List<Object> varTypes = new ArrayList<>();
                     varTypes.add(NonTaggedFormatUtil.getTokenType(secondaryKeyType));
 
                     // If the index is a length-partitioned, then create
@@ -433,11 +448,11 @@
                     tokenUpdate.getInputs().add(new MutableObject<ILogicalOperator>(assign));
                     context.computeAndSetTypeEnvironmentForOperator(tokenUpdate);
 
-                    IndexInsertDeleteUpsertOperator indexUpdate =
-                            new IndexInsertDeleteUpsertOperator(dataSourceIndex, insertOp.getPrimaryKeyExpressions(),
-                                    tokenizeKeyExprs, filterExpression, insertOp.getOperation(), insertOp.isBulkload(),
-                                    insertOp.getAdditionalNonFilteringExpressions() == null ? 0
-                                            : insertOp.getAdditionalNonFilteringExpressions().size());
+                    IndexInsertDeleteUpsertOperator indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
+                            insertOp.getPrimaryKeyExpressions(), tokenizeKeyExprs, filterExpression,
+                            insertOp.getOperation(), insertOp.isBulkload(),
+                            insertOp.getAdditionalNonFilteringExpressions() == null ? 0
+                                    : insertOp.getAdditionalNonFilteringExpressions().size());
                     indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                     indexUpdate.getInputs().add(new MutableObject<ILogicalOperator>(tokenUpdate));
 
@@ -448,11 +463,11 @@
 
                 } else {
                     // When TokenizeOperator is not needed
-                    IndexInsertDeleteUpsertOperator indexUpdate =
-                            new IndexInsertDeleteUpsertOperator(dataSourceIndex, insertOp.getPrimaryKeyExpressions(),
-                                    secondaryExpressions, filterExpression, insertOp.getOperation(),
-                                    insertOp.isBulkload(), insertOp.getAdditionalNonFilteringExpressions() == null ? 0
-                                            : insertOp.getAdditionalNonFilteringExpressions().size());
+                    IndexInsertDeleteUpsertOperator indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
+                            insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
+                            insertOp.getOperation(), insertOp.isBulkload(),
+                            insertOp.getAdditionalNonFilteringExpressions() == null ? 0
+                                    : insertOp.getAdditionalNonFilteringExpressions().size());
 
                     indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                     // We add the necessary expressions for upsert
@@ -476,16 +491,16 @@
 
             } else if (index.getIndexType() == IndexType.RTREE) {
                 // Get type, dimensions and number of keys
-                Pair<IAType, Boolean> keyPairType =
-                        Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0), secondaryKeyFields.get(0), recType);
+                Pair<IAType, Boolean> keyPairType = Index.getNonNullableOpenFieldType(secondaryKeyTypes.get(0),
+                        secondaryKeyFields.get(0), recType);
                 IAType spatialType = keyPairType.first;
-                boolean isPointMBR =
-                        spatialType.getTypeTag() == ATypeTag.POINT || spatialType.getTypeTag() == ATypeTag.POINT3D;
+                boolean isPointMBR = spatialType.getTypeTag() == ATypeTag.POINT
+                        || spatialType.getTypeTag() == ATypeTag.POINT3D;
                 int dimension = NonTaggedFormatUtil.getNumDimensions(spatialType.getTypeTag());
                 int numKeys = (isPointMBR && isBulkload) ? dimension : dimension * 2;
                 // Get variables and expressions
-                List<LogicalVariable> keyVarList = new ArrayList<LogicalVariable>();
-                List<Mutable<ILogicalExpression>> keyExprList = new ArrayList<Mutable<ILogicalExpression>>();
+                List<LogicalVariable> keyVarList = new ArrayList<>();
+                List<Mutable<ILogicalExpression>> keyExprList = new ArrayList<>();
                 for (int i = 0; i < numKeys; i++) {
                     LogicalVariable keyVar = context.newVar();
                     keyVarList.add(keyVar);
@@ -518,7 +533,7 @@
                 AssignOperator originalAssignCoordinates = null;
                 // We do something similar for previous key if the operation is an upsert
                 if (insertOp.getOperation() == Kind.UPSERT) {
-                    List<LogicalVariable> originalKeyVarList = new ArrayList<LogicalVariable>();
+                    List<LogicalVariable> originalKeyVarList = new ArrayList<>();
                     List<Mutable<ILogicalExpression>> originalKeyExprList = new ArrayList<>();
                     // we don't do any filtering since nulls are expected here and there
                     for (int i = 0; i < numKeys; i++) {
@@ -557,11 +572,11 @@
                             context.getOutputTypeEnvironment(assignCoordinates), forceFilter);
                 }
                 AqlIndex dataSourceIndex = new AqlIndex(index, dataverseName, datasetName, mp);
-                IndexInsertDeleteUpsertOperator indexUpdate =
-                        new IndexInsertDeleteUpsertOperator(dataSourceIndex, insertOp.getPrimaryKeyExpressions(),
-                                secondaryExpressions, filterExpression, insertOp.getOperation(), insertOp.isBulkload(),
-                                insertOp.getAdditionalNonFilteringExpressions() == null ? 0
-                                        : insertOp.getAdditionalNonFilteringExpressions().size());
+                IndexInsertDeleteUpsertOperator indexUpdate = new IndexInsertDeleteUpsertOperator(dataSourceIndex,
+                        insertOp.getPrimaryKeyExpressions(), secondaryExpressions, filterExpression,
+                        insertOp.getOperation(), insertOp.isBulkload(),
+                        insertOp.getAdditionalNonFilteringExpressions() == null ? 0
+                                : insertOp.getAdditionalNonFilteringExpressions().size());
                 indexUpdate.setAdditionalFilteringExpressions(filteringExpressions);
                 if (insertOp.getOperation() == Kind.UPSERT) {
                     // set old secondary key expressions
@@ -601,12 +616,21 @@
         return true;
     }
 
+    private void connectCastOperator(IOptimizationContext context, ILogicalOperator currentTop,
+            AssignOperator castedRecordAssignOperator) throws AlgebricksException {
+        castedRecordAssignOperator.getInputs().addAll(currentTop.getInputs());
+        currentTop.getInputs().clear();
+        currentTop.getInputs().add(new MutableObject<>(castedRecordAssignOperator));
+        context.computeAndSetTypeEnvironmentForOperator(castedRecordAssignOperator);
+        context.computeAndSetTypeEnvironmentForOperator(currentTop);
+    }
+
     // Merges typed index fields with specified recordType, allowing indexed fields to be optional.
     // I.e. the type { "personId":int32, "name": string, "address" : { "street": string } } with typed indexes on age:int32, address.state:string
     //      will be merged into type { "personId":int32, "name": string, "age": int32? "address" : { "street": string, "state": string? } }
     // Used by open indexes to enforce the type of an indexed record
     public static ARecordType createEnforcedType(ARecordType initialType, List<Index> indexes)
-            throws AsterixException, AlgebricksException {
+            throws AlgebricksException {
         ARecordType enforcedType = initialType;
         for (Index index : indexes) {
             if (!index.isSecondaryIndex() || !index.isEnforcingKeyFileds()) {
@@ -616,7 +640,7 @@
                 throw new AlgebricksException("Indexing an open field is only supported on the record part");
             }
             for (int i = 0; i < index.getKeyFieldNames().size(); i++) {
-                Stack<Pair<ARecordType, String>> nestedTypeStack = new Stack<Pair<ARecordType, String>>();
+                Stack<Pair<ARecordType, String>> nestedTypeStack = new Stack<>();
                 List<String> splits = index.getKeyFieldNames().get(i);
                 ARecordType nestedFieldType = enforcedType;
                 boolean openRecords = false;
@@ -632,7 +656,7 @@
                         break;
                     }
                 }
-                if (openRecords == true) {
+                if (openRecords) {
                     // create the smallest record
                     enforcedType = new ARecordType(splits.get(splits.size() - 2),
                             new String[] { splits.get(splits.size() - 1) },
@@ -676,8 +700,8 @@
                             nestedFieldType.isOpen());
                 }
 
-                // Create the enforcedtype for the nested fields in the schema, from the ground up
-                if (nestedTypeStack.size() > 0) {
+                // Create the enforced type for the nested fields in the schema, from the ground up
+                if (!nestedTypeStack.empty()) {
                     while (!nestedTypeStack.isEmpty()) {
                         Pair<ARecordType, String> nestedTypePair = nestedTypeStack.pop();
                         ARecordType nestedRecType = nestedTypePair.first;
@@ -716,7 +740,6 @@
      * @param context
      * @throws AlgebricksException
      */
-    @SuppressWarnings("unchecked")
     private void prepareVarAndExpression(List<String> fields, String[] recordFields, LogicalVariable recordVar,
             List<Mutable<ILogicalExpression>> expressions, List<LogicalVariable> vars, IOptimizationContext context)
             throws AlgebricksException {
@@ -740,13 +763,13 @@
                 for (int i = 0; i < fields.size(); i++) {
                     fieldList.add(new AString(fields.get(i)));
                 }
-                Mutable<ILogicalExpression> fieldRef = new MutableObject<ILogicalExpression>(
+                Mutable<ILogicalExpression> fieldRef = new MutableObject<>(
                         new ConstantExpression(new AsterixConstantValue(fieldList)));
                 // Create an expression for the nested case
                 func = new ScalarFunctionCallExpression(
                         FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_NESTED), varRef, fieldRef);
             } else {
-                Mutable<ILogicalExpression> fieldRef = new MutableObject<ILogicalExpression>(
+                Mutable<ILogicalExpression> fieldRef = new MutableObject<>(
                         new ConstantExpression(new AsterixConstantValue(new AString(fields.get(0)))));
                 // Create an expression for the open field case (By name)
                 func = new ScalarFunctionCallExpression(
@@ -754,7 +777,7 @@
             }
         } else {
             // Assumes the indexed field is in the closed portion of the type.
-            Mutable<ILogicalExpression> indexRef = new MutableObject<ILogicalExpression>(
+            Mutable<ILogicalExpression> indexRef = new MutableObject<>(
                     new ConstantExpression(new AsterixConstantValue(new AInt32(pos))));
             func = new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.FIELD_ACCESS_BY_INDEX), varRef, indexRef);
@@ -764,10 +787,9 @@
         vars.add(newVar);
     }
 
-    @SuppressWarnings("unchecked")
     private Mutable<ILogicalExpression> createFilterExpression(List<LogicalVariable> secondaryKeyVars,
             IVariableTypeEnvironment typeEnv, boolean forceFilter) throws AlgebricksException {
-        List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<Mutable<ILogicalExpression>>();
+        List<Mutable<ILogicalExpression>> filterExpressions = new ArrayList<>();
         // Add 'is not null' to all nullable secondary index keys as a filtering
         // condition.
         for (LogicalVariable secondaryKeyVar : secondaryKeyVars) {
@@ -775,22 +797,22 @@
             if (!NonTaggedFormatUtil.isOptional(secondaryKeyType) && !forceFilter) {
                 continue;
             }
-            ScalarFunctionCallExpression isUnknownFuncExpr =
-                    new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.IS_UNKOWN),
-                            new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
-            ScalarFunctionCallExpression notFuncExpr =
-                    new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.NOT),
-                            new MutableObject<ILogicalExpression>(isUnknownFuncExpr));
+            ScalarFunctionCallExpression isUnknownFuncExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.IS_UNKOWN),
+                    new MutableObject<ILogicalExpression>(new VariableReferenceExpression(secondaryKeyVar)));
+            ScalarFunctionCallExpression notFuncExpr = new ScalarFunctionCallExpression(
+                    FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.NOT),
+                    new MutableObject<ILogicalExpression>(isUnknownFuncExpr));
             filterExpressions.add(new MutableObject<ILogicalExpression>(notFuncExpr));
         }
         // No nullable secondary keys.
         if (filterExpressions.isEmpty()) {
             return null;
         }
-        Mutable<ILogicalExpression> filterExpression = null;
+        Mutable<ILogicalExpression> filterExpression;
         if (filterExpressions.size() > 1) {
             // Create a conjunctive condition.
-            filterExpression = new MutableObject<ILogicalExpression>(new ScalarFunctionCallExpression(
+            filterExpression = new MutableObject<>(new ScalarFunctionCallExpression(
                     FunctionUtil.getFunctionInfo(AsterixBuiltinFunctions.AND), filterExpressions));
         } else {
             filterExpression = filterExpressions.get(0);
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.aql
new file mode 100644
index 0000000..92d8f7d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.aql
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type OrderOpenType as open {
+  o_orderkey: int64
+}
+
+create dataset OrdersOpen(OrderOpenType)
+primary key o_orderkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.ddl.aql
new file mode 100644
index 0000000..ad05499
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.ddl.aql
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use dataverse test;
+
+insert into dataset OrdersOpen (
+  {"o_orderkey": 1,
+  "o_custkey": 1}
+)
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.aql
new file mode 100644
index 0000000..66d6fdb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use dataverse test;
+
+create index idx_Orders_Custkey on 
+OrdersOpen(o_custkey:int32) enforced;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.ddl.aql
new file mode 100644
index 0000000..9ac581e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.ddl.aql
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use dataverse test;
+
+delete $v from dataset OrdersOpen
+where $v. o_orderkey = 1;
+
+insert into dataset OrdersOpen (
+  {"o_orderkey": 1,
+  "o_custkey": 2}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.aql
new file mode 100644
index 0000000..b31dd8e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.aql
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Delete from enforced index and validate deletion
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use dataverse test;
+
+let $l :=  for $o in dataset('OrdersOpen') 
+where $o.o_custkey >=-1
+return $o.o_orderKey
+return  count($l);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.aql
new file mode 100644
index 0000000..f710221
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.aql
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+drop dataverse test if exists;
+create dataverse test;
+use dataverse test;
+
+create type OrderOpenType as open {
+  o_orderkey: int64
+}
+
+create dataset OrdersOpen(OrderOpenType)
+primary key o_orderkey;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.ddl.aql
new file mode 100644
index 0000000..517004d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.ddl.aql
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use dataverse test;
+
+insert into dataset OrdersOpen (
+  {"o_orderkey": 1,
+  "o_custkey": 1}
+)
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.aql
new file mode 100644
index 0000000..9b2d138
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.aql
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use dataverse test;
+
+create index idx_Orders_Custkey on 
+OrdersOpen(o_custkey:int32) enforced;
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.ddl.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.ddl.aql
new file mode 100644
index 0000000..cbc4459
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.ddl.aql
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use dataverse test;
+
+upsert into dataset OrdersOpen (
+  {"o_orderkey": 1,
+  "o_custkey": 2}
+);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.aql
new file mode 100644
index 0000000..7d54933
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.aql
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+* Description  : Upsert from enforced index and validate result
+* Expected Res : Success
+* Date         : 22 Aug 2016
+*/
+use dataverse test;
+
+let $l :=  for $o in dataset('OrdersOpen') 
+where $o.o_custkey >=-1
+return $o.o_orderKey
+return  count($l);
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.adm
new file mode 100644
index 0000000..56a6051
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.adm
@@ -0,0 +1 @@
+1
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.adm
new file mode 100644
index 0000000..56a6051
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.adm
@@ -0,0 +1 @@
+1
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
index e5710bc..27141a2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
@@ -3495,6 +3495,18 @@
         </compilation-unit>
       </test-case>
     </test-group>
+    <test-group name="open-index-enforced/type-checking">
+      <test-case FilePath="open-index-enforced/type-checking">
+        <compilation-unit name="enforced-type-delete">
+          <output-dir compare="Text">enforced-type-delete</output-dir>
+        </compilation-unit>
+      </test-case>
+      <test-case FilePath="open-index-enforced/type-checking">
+        <compilation-unit name="enforced-type-upsert">
+          <output-dir compare="Text">enforced-type-upsert</output-dir>
+        </compilation-unit>
+      </test-case>
+    </test-group>
   </test-group>
   <test-group name="nested-open-index">
     <test-group name="nested-open-index/index-join">

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newchange
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>


Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Murtadha Hubail has abandoned this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Abandoned

Replaced by https://asterix-gerrit.ics.uci.edu/#/c/1146/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: abandon
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2357/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2356/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1109

to look at the new patch set (#3).

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................

ASTERIXDB-1451: Fix type propagation to enforced indexes

This change includes the following:
- Introduce cast function in case of delete operation
  after the primary index to ensure types are passed
  correctly to enforced indexes.
- Introduce cast function in case of upsert operation
  before old secondary keys extraction to ensure types
  are passed correctly to enforced indexes.
- Sonar-Qube fixes.

Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
14 files changed, 415 insertions(+), 82 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/09/1109/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/431/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

I'd like you to reexamine a change.  Please visit

    https://asterix-gerrit.ics.uci.edu/1109

to look at the new patch set (#2).

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................

ASTERIXDB-1451: Fix type propagation to enforced indexes

This change includes the following:
- Introduce cast function in case of delete operation
  after the primary index to ensure types are passed
  correctly to enforced indexes.
- Introduce cast function in case of upsert operation
  before old secondary keys extraction to ensure types
  are passed correctly to enforced indexes.
- Sonar-Qube fixes.

Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.2.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.3.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.4.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.1.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.2.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.3.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.4.ddl.aql
A asterixdb/asterix-app/src/test/resources/runtimets/queries/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.query.aql
A asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-delete/enforced-type-delete.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/open-index-enforced/type-checking/enforced-type-upsert/enforced-type-upsert.5.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite.xml
14 files changed, 415 insertions(+), 82 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/09/1109/2
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 3:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/431/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/2355/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/430/ : SUCCESS

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 3: -Code-Review

Can you please abandon this change since it has been replaced with:
https://asterix-gerrit.ics.uci.edu/#/c/1146/

Thanks,
~Abdullah.

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 3: Code-Review+2

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/430/

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: ASTERIXDB-1451: Fix type propagation to enforced indexes

Posted by "Yingyi Bu (Code Review)" <do...@asterixdb.incubator.apache.org>.
Yingyi Bu has posted comments on this change.

Change subject: ASTERIXDB-1451: Fix type propagation to enforced indexes
......................................................................


Patch Set 3:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1109/3/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java:

Line 280:                                 .add(new MutableObject<>(new VariableReferenceExpression(insertOp.getPrevRecordVar())));
For the newly added code (Upsert), can we cast the individual indexed field(s) instead of the entire record, as the function CastType works for any non-record types as well?

For the current open index implementation, it's a big waste to cast the entire record: https://issues.apache.org/jira/browse/ASTERIXDB-1404.

I have unified different cast functions into cast-type, which can cast any input typed item into any required type, not necessarily a record.


Line 643:                 Stack<Pair<ARecordType, String>> nestedTypeStack = new Stack<>();
> MAJOR SonarQube violation:
This should be fixed?


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1109
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: comment
Gerrit-Change-Id: I1b4572ca541e6a9fbaf1d7f107f76630c854abdd
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes