You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu> on 2021/06/02 20:17:58 UTC

Change in asterixdb[master]: PLEASE EDIT to provide a meaningful commit message!

From Glenn Galvizo <gg...@uci.edu>:

Glenn Galvizo has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723 )


Change subject: PLEASE EDIT to provide a meaningful commit message!
......................................................................

PLEASE EDIT to provide a meaningful commit message!

The following commits from your working branch will be included:

commit 8cb9a782df3ec1c8e42a59341d7d7b37d0eb5906
Author: ggalvizo <gg...@uci.edu>
Date:   Sat May 29 19:43:34 2021 -0700

    Currently using the try-catch pattern to implement the implicit DISTINCT for array index maintenance.

Change-Id: I328bf4e80037a23b6d44139548436f6c4f232bba
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
3 files changed, 66 insertions(+), 49 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/23/11723/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 c3859be..182687d 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
@@ -449,7 +449,7 @@
                         context.computeAndSetTypeEnvironmentForOperator(unnestSourceOp);
                         UnnestBranchCreator unnestSIDXBranch = buildUnnestBranch(unnestSourceOp, index, newRecordVar,
                                 newMetaVar, recType, metaType, dataset.hasMetaPart());
-                        unnestSIDXBranch.applyProjectDistinct();
+                        unnestSIDXBranch.applyProjectOnly();
 
                         // If there exists a filter expression, add it to the top of our nested plan.
                         filterExpression = (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
@@ -477,7 +477,7 @@
                             UnnestBranchCreator unnestBeforeSIDXBranch = buildUnnestBranch(unnestBeforeSourceOp, index,
                                     primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, recType,
                                     metaType, dataset.hasMetaPart());
-                            unnestBeforeSIDXBranch.applyProjectDistinct();
+                            unnestBeforeSIDXBranch.applyProjectOnly();
                             indexUpdate.getNestedPlans().add(unnestBeforeSIDXBranch.buildBranch());
                         }
                     } else if (index.getIndexType() == IndexType.ARRAY && isBulkload) {
@@ -1010,6 +1010,13 @@
             return varRef;
         }
 
+        public final void applyProjectOnly() throws AlgebricksException {
+            List<LogicalVariable> projectVars = new ArrayList<>(this.lastFieldVars);
+            ProjectOperator projectOperator = new ProjectOperator(projectVars);
+            projectOperator.setSourceLocation(sourceLoc);
+            this.currentTop = introduceNewOp(currentTop, projectOperator, true);
+        }
+
         @SafeVarargs
         public final void applyProjectDistinct(List<Mutable<ILogicalExpression>>... auxiliaryExpressions)
                 throws AlgebricksException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
index 0b36774..99b9a08 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
@@ -31,13 +31,13 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
 
@@ -122,10 +122,8 @@
     private class IndexTupleInsertDelete implements IFrameWriter {
         private final RecordDescriptor inputRecordDescriptor;
         private FrameTupleAccessor endOfPipelineTupleAccessor;
-
-        // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
-        private ArrayTupleBuilder arrayTupleBuilder;
-        private ArrayTupleReference arrayTupleReference;
+        private FrameTupleReference endOfPipelineTupleReference;
+        private ConcatenatingTupleReference endTupleReference;
 
         private IndexTupleInsertDelete(RecordDescriptor recordDescriptor) {
             this.inputRecordDescriptor = recordDescriptor;
@@ -133,11 +131,11 @@
 
         @Override
         public void open() throws HyracksDataException {
-            int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+            int indexEntryLength = inputRecordDescriptor.getFieldCount() + numberOfPrimaryKeyAndFilterFields;
+            endTupleReference = new ConcatenatingTupleReference(indexEntryLength);
 
             endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
-            arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
-            arrayTupleReference = new ArrayTupleReference();
+            endOfPipelineTupleReference = new FrameTupleReference();
         }
 
         @Override
@@ -147,25 +145,35 @@
             endOfPipelineTupleAccessor.reset(buffer);
             int nTuple = endOfPipelineTupleAccessor.getTupleCount();
             for (int t = 0; t < nTuple; t++) {
+                endOfPipelineTupleReference.reset(endOfPipelineTupleAccessor, t);
+                endTupleReference.reset();
 
-                // First, add the secondary keys.
-                arrayTupleBuilder.reset();
-                int nFields = endOfPipelineTupleAccessor.getFieldCount();
-                for (int f = 0; f < nFields; f++) {
-                    arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
-                }
+                // Add the secondary keys.
+                endTupleReference.addTuple(endOfPipelineTupleReference);
 
-                // Next, add the primary keys and filter fields.
-                for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
-                    arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
-                }
+                // Add the primary keys and filter fields.
+                endTupleReference.addTuple(tuple);
 
-                // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
-                arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
+                // Pass the tuple to our accessor. There are only two operations: insert or delete.
                 if (op.equals(IndexOperation.INSERT)) {
-                    workingLSMAccessor.forceInsert(arrayTupleReference);
+                    try {
+                        workingLSMAccessor.forceInsert(endTupleReference);
+                    } catch (HyracksDataException e) {
+                        if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.DUPLICATE_KEY)) {
+                            throw e;
+                        }
+                    }
+// Showing that I am PhD quality, changing directions for PhD to graph database, number of minutes per slide
+//
+
                 } else {
-                    workingLSMAccessor.forceDelete(arrayTupleReference);
+                    try {
+                        workingLSMAccessor.forceDelete(endTupleReference);
+                    } catch (HyracksDataException e) {
+                        if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
+                            throw e;
+                        }
+                    }
                 }
             }
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
index f1af496..91f3920 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
@@ -33,13 +33,12 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 
 public class LSMSecondaryUpsertWithNestedPlanOperatorNodePushable extends LSMSecondaryUpsertOperatorNodePushable {
@@ -154,10 +153,7 @@
 
         private FrameTupleAccessor endOfPipelineTupleAccessor;
         private FrameTupleReference endOfPipelineTupleReference;
-
-        // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
-        private ArrayTupleBuilder arrayTupleBuilder;
-        private ArrayTupleReference arrayTupleReference;
+        private ConcatenatingTupleReference endTupleReference;
 
         private IndexTupleUnconditionalOperation(RecordDescriptor recordDescriptor, boolean isInsert) {
             this.inputRecordDescriptor = recordDescriptor;
@@ -166,11 +162,11 @@
 
         @Override
         public void open() throws HyracksDataException {
-            int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+            int indexEntryLength = inputRecordDescriptor.getFieldCount() + numberOfPrimaryKeyAndFilterFields;
+            endTupleReference = new ConcatenatingTupleReference(indexEntryLength);
+
             endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
             endOfPipelineTupleReference = new FrameTupleReference();
-            arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
-            arrayTupleReference = new ArrayTupleReference();
         }
 
         @Override
@@ -180,33 +176,39 @@
             endOfPipelineTupleAccessor.reset(buffer);
             int nTuple = endOfPipelineTupleAccessor.getTupleCount();
             for (int t = 0; t < nTuple; t++) {
-
                 endOfPipelineTupleReference.reset(endOfPipelineTupleAccessor, t);
+
+                // Do not perform operations w/ null or missing values (same behavior as atomic upserts).
                 if (hasNullOrMissing(endOfPipelineTupleReference)) {
-                    // Do not perform operations w/ null or missing values (same behavior as atomic upserts).
                     continue;
                 }
 
-                // First, add the secondary keys.
-                arrayTupleBuilder.reset();
-                int nFields = endOfPipelineTupleAccessor.getFieldCount();
-                for (int f = 0; f < nFields; f++) {
-                    arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
-                }
+                // Add the secondary keys.
+                endTupleReference.reset();
+                endTupleReference.addTuple(endOfPipelineTupleReference);
 
-                // Next, add the primary keys and filter fields.
-                for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
-                    arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
-                }
+                // Add the primary keys and filter fields.
+                endTupleReference.addTuple(tuple);
 
                 // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
-                arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
                 if (this.isInsert) {
                     abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.INSERT);
-                    workingLSMAccessor.forceInsert(arrayTupleReference);
+                    try {
+                        workingLSMAccessor.forceInsert(endTupleReference);
+                    } catch (HyracksDataException e) {
+                        if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.DUPLICATE_KEY)) {
+                            throw e;
+                        }
+                    }
                 } else {
                     abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.DELETE);
-                    workingLSMAccessor.forceDelete(arrayTupleReference);
+                    try {
+                        workingLSMAccessor.forceDelete(endTupleReference);
+                    } catch (HyracksDataException e) {
+                        if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
+                            throw e;
+                        }
+                    }
                 }
             }
         }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I328bf4e80037a23b6d44139548436f6c4f232bba
Gerrit-Change-Number: 11723
Gerrit-PatchSet: 1
Gerrit-Owner: Glenn Galvizo <gg...@uci.edu>
Gerrit-MessageType: newchange

Change in asterixdb[master]: PLEASE EDIT to provide a meaningful commit message!

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723 )

Change subject: PLEASE EDIT to provide a meaningful commit message!
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/zYbieJQarZH3SyXt5 : SUCCESS


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I328bf4e80037a23b6d44139548436f6c4f232bba
Gerrit-Change-Number: 11723
Gerrit-PatchSet: 1
Gerrit-Owner: Glenn Galvizo <gg...@uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-CC: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 02 Jun 2021 20:27:36 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[master]: PLEASE EDIT to provide a meaningful commit message!

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Glenn Galvizo <gg...@uci.edu>:

Glenn Galvizo has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723 )


Change subject: PLEASE EDIT to provide a meaningful commit message!
......................................................................

PLEASE EDIT to provide a meaningful commit message!

The following commits from your working branch will be included:

commit 8cb9a782df3ec1c8e42a59341d7d7b37d0eb5906
Author: ggalvizo <gg...@uci.edu>
Date:   Sat May 29 19:43:34 2021 -0700

    Currently using the try-catch pattern to implement the implicit DISTINCT for array index maintenance.

Change-Id: I328bf4e80037a23b6d44139548436f6c4f232bba
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/IntroduceSecondaryIndexInsertDeleteRule.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
3 files changed, 66 insertions(+), 49 deletions(-)



  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/23/11723/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 c3859be..182687d 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
@@ -449,7 +449,7 @@
                         context.computeAndSetTypeEnvironmentForOperator(unnestSourceOp);
                         UnnestBranchCreator unnestSIDXBranch = buildUnnestBranch(unnestSourceOp, index, newRecordVar,
                                 newMetaVar, recType, metaType, dataset.hasMetaPart());
-                        unnestSIDXBranch.applyProjectDistinct();
+                        unnestSIDXBranch.applyProjectOnly();
 
                         // If there exists a filter expression, add it to the top of our nested plan.
                         filterExpression = (primaryIndexModificationOp.getOperation() == Kind.UPSERT) ? null
@@ -477,7 +477,7 @@
                             UnnestBranchCreator unnestBeforeSIDXBranch = buildUnnestBranch(unnestBeforeSourceOp, index,
                                     primaryIndexModificationOp.getBeforeOpRecordVar(), beforeOpMetaVar, recType,
                                     metaType, dataset.hasMetaPart());
-                            unnestBeforeSIDXBranch.applyProjectDistinct();
+                            unnestBeforeSIDXBranch.applyProjectOnly();
                             indexUpdate.getNestedPlans().add(unnestBeforeSIDXBranch.buildBranch());
                         }
                     } else if (index.getIndexType() == IndexType.ARRAY && isBulkload) {
@@ -1010,6 +1010,13 @@
             return varRef;
         }
 
+        public final void applyProjectOnly() throws AlgebricksException {
+            List<LogicalVariable> projectVars = new ArrayList<>(this.lastFieldVars);
+            ProjectOperator projectOperator = new ProjectOperator(projectVars);
+            projectOperator.setSourceLocation(sourceLoc);
+            this.currentTop = introduceNewOp(currentTop, projectOperator, true);
+        }
+
         @SafeVarargs
         public final void applyProjectDistinct(List<Mutable<ILogicalExpression>>... auxiliaryExpressions)
                 throws AlgebricksException {
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
index 0b36774..99b9a08 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryInsertDeleteWithNestedPlanOperatorNodePushable.java
@@ -31,13 +31,13 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
+import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
 import org.apache.hyracks.storage.am.common.ophelpers.IndexOperation;
+import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import org.apache.hyracks.storage.am.lsm.common.dataflow.LSMIndexInsertUpdateDeleteOperatorNodePushable;
 
@@ -122,10 +122,8 @@
     private class IndexTupleInsertDelete implements IFrameWriter {
         private final RecordDescriptor inputRecordDescriptor;
         private FrameTupleAccessor endOfPipelineTupleAccessor;
-
-        // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
-        private ArrayTupleBuilder arrayTupleBuilder;
-        private ArrayTupleReference arrayTupleReference;
+        private FrameTupleReference endOfPipelineTupleReference;
+        private ConcatenatingTupleReference endTupleReference;
 
         private IndexTupleInsertDelete(RecordDescriptor recordDescriptor) {
             this.inputRecordDescriptor = recordDescriptor;
@@ -133,11 +131,11 @@
 
         @Override
         public void open() throws HyracksDataException {
-            int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+            int indexEntryLength = inputRecordDescriptor.getFieldCount() + numberOfPrimaryKeyAndFilterFields;
+            endTupleReference = new ConcatenatingTupleReference(indexEntryLength);
 
             endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
-            arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
-            arrayTupleReference = new ArrayTupleReference();
+            endOfPipelineTupleReference = new FrameTupleReference();
         }
 
         @Override
@@ -147,25 +145,35 @@
             endOfPipelineTupleAccessor.reset(buffer);
             int nTuple = endOfPipelineTupleAccessor.getTupleCount();
             for (int t = 0; t < nTuple; t++) {
+                endOfPipelineTupleReference.reset(endOfPipelineTupleAccessor, t);
+                endTupleReference.reset();
 
-                // First, add the secondary keys.
-                arrayTupleBuilder.reset();
-                int nFields = endOfPipelineTupleAccessor.getFieldCount();
-                for (int f = 0; f < nFields; f++) {
-                    arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
-                }
+                // Add the secondary keys.
+                endTupleReference.addTuple(endOfPipelineTupleReference);
 
-                // Next, add the primary keys and filter fields.
-                for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
-                    arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
-                }
+                // Add the primary keys and filter fields.
+                endTupleReference.addTuple(tuple);
 
-                // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
-                arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
+                // Pass the tuple to our accessor. There are only two operations: insert or delete.
                 if (op.equals(IndexOperation.INSERT)) {
-                    workingLSMAccessor.forceInsert(arrayTupleReference);
+                    try {
+                        workingLSMAccessor.forceInsert(endTupleReference);
+                    } catch (HyracksDataException e) {
+                        if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.DUPLICATE_KEY)) {
+                            throw e;
+                        }
+                    }
+// Showing that I am PhD quality, changing directions for PhD to graph database, number of minutes per slide
+//
+
                 } else {
-                    workingLSMAccessor.forceDelete(arrayTupleReference);
+                    try {
+                        workingLSMAccessor.forceDelete(endTupleReference);
+                    } catch (HyracksDataException e) {
+                        if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
+                            throw e;
+                        }
+                    }
                 }
             }
         }
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
index f1af496..91f3920 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/operators/LSMSecondaryUpsertWithNestedPlanOperatorNodePushable.java
@@ -33,13 +33,12 @@
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
-import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
 import org.apache.hyracks.dataflow.common.comm.io.FrameTupleAccessor;
 import org.apache.hyracks.dataflow.common.comm.util.FrameUtils;
 import org.apache.hyracks.dataflow.common.data.accessors.FrameTupleReference;
 import org.apache.hyracks.storage.am.common.api.IModificationOperationCallbackFactory;
 import org.apache.hyracks.storage.am.common.dataflow.IIndexDataflowHelperFactory;
+import org.apache.hyracks.storage.am.common.tuples.ConcatenatingTupleReference;
 import org.apache.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 
 public class LSMSecondaryUpsertWithNestedPlanOperatorNodePushable extends LSMSecondaryUpsertOperatorNodePushable {
@@ -154,10 +153,7 @@
 
         private FrameTupleAccessor endOfPipelineTupleAccessor;
         private FrameTupleReference endOfPipelineTupleReference;
-
-        // We are not writing the resulting tuple to a frame, we must store the result in an intermediate.
-        private ArrayTupleBuilder arrayTupleBuilder;
-        private ArrayTupleReference arrayTupleReference;
+        private ConcatenatingTupleReference endTupleReference;
 
         private IndexTupleUnconditionalOperation(RecordDescriptor recordDescriptor, boolean isInsert) {
             this.inputRecordDescriptor = recordDescriptor;
@@ -166,11 +162,11 @@
 
         @Override
         public void open() throws HyracksDataException {
-            int numSecondaryKeys = inputRecordDescriptor.getFieldCount();
+            int indexEntryLength = inputRecordDescriptor.getFieldCount() + numberOfPrimaryKeyAndFilterFields;
+            endTupleReference = new ConcatenatingTupleReference(indexEntryLength);
+
             endOfPipelineTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
             endOfPipelineTupleReference = new FrameTupleReference();
-            arrayTupleBuilder = new ArrayTupleBuilder(numberOfPrimaryKeyAndFilterFields + numSecondaryKeys);
-            arrayTupleReference = new ArrayTupleReference();
         }
 
         @Override
@@ -180,33 +176,39 @@
             endOfPipelineTupleAccessor.reset(buffer);
             int nTuple = endOfPipelineTupleAccessor.getTupleCount();
             for (int t = 0; t < nTuple; t++) {
-
                 endOfPipelineTupleReference.reset(endOfPipelineTupleAccessor, t);
+
+                // Do not perform operations w/ null or missing values (same behavior as atomic upserts).
                 if (hasNullOrMissing(endOfPipelineTupleReference)) {
-                    // Do not perform operations w/ null or missing values (same behavior as atomic upserts).
                     continue;
                 }
 
-                // First, add the secondary keys.
-                arrayTupleBuilder.reset();
-                int nFields = endOfPipelineTupleAccessor.getFieldCount();
-                for (int f = 0; f < nFields; f++) {
-                    arrayTupleBuilder.addField(endOfPipelineTupleAccessor, t, f);
-                }
+                // Add the secondary keys.
+                endTupleReference.reset();
+                endTupleReference.addTuple(endOfPipelineTupleReference);
 
-                // Next, add the primary keys and filter fields.
-                for (int f = 0; f < numberOfPrimaryKeyAndFilterFields; f++) {
-                    arrayTupleBuilder.addField(tuple.getFieldData(f), tuple.getFieldStart(f), tuple.getFieldLength(f));
-                }
+                // Add the primary keys and filter fields.
+                endTupleReference.addTuple(tuple);
 
                 // Finally, pass the tuple to our accessor. There are only two operations: insert or delete.
-                arrayTupleReference.reset(arrayTupleBuilder.getFieldEndOffsets(), arrayTupleBuilder.getByteArray());
                 if (this.isInsert) {
                     abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.INSERT);
-                    workingLSMAccessor.forceInsert(arrayTupleReference);
+                    try {
+                        workingLSMAccessor.forceInsert(endTupleReference);
+                    } catch (HyracksDataException e) {
+                        if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.DUPLICATE_KEY)) {
+                            throw e;
+                        }
+                    }
                 } else {
                     abstractModCallback.setOp(AbstractIndexModificationOperationCallback.Operation.DELETE);
-                    workingLSMAccessor.forceDelete(arrayTupleReference);
+                    try {
+                        workingLSMAccessor.forceDelete(endTupleReference);
+                    } catch (HyracksDataException e) {
+                        if (!e.matches(org.apache.hyracks.api.exceptions.ErrorCode.UPDATE_OR_DELETE_NON_EXISTENT_KEY)) {
+                            throw e;
+                        }
+                    }
                 }
             }
         }

-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I328bf4e80037a23b6d44139548436f6c4f232bba
Gerrit-Change-Number: 11723
Gerrit-PatchSet: 1
Gerrit-Owner: Glenn Galvizo <gg...@uci.edu>
Gerrit-MessageType: newchange

Change in asterixdb[master]: [NO-ISSUE][IDX] Using an implicit DISTINCT for array index maintenance.

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
Anon. E. Moose #1000171 has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723 )

Change subject: [NO-ISSUE][IDX] Using an implicit DISTINCT for array index maintenance.
......................................................................


Patch Set 3: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/5jvFgsGHLBYXdqZ46 : UNSTABLE


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I328bf4e80037a23b6d44139548436f6c4f232bba
Gerrit-Change-Number: 11723
Gerrit-PatchSet: 3
Gerrit-Owner: Glenn Galvizo <gg...@uci.edu>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Thu, 24 Jun 2021 21:17:37 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[master]: [NO-ISSUE][IDX] Using an implicit DISTINCT for array index maintenance.

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Glenn Galvizo <gg...@uci.edu>:

Glenn Galvizo has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723 )

Change subject: [NO-ISSUE][IDX] Using an implicit DISTINCT for array index maintenance.
......................................................................


Patch Set 4: Code-Review+2


-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/11723
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Change-Id: I328bf4e80037a23b6d44139548436f6c4f232bba
Gerrit-Change-Number: 11723
Gerrit-PatchSet: 4
Gerrit-Owner: Glenn Galvizo <gg...@uci.edu>
Gerrit-Reviewer: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Glenn Galvizo <gg...@uci.edu>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Wed, 30 Jun 2021 21:42:24 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment