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 2022/01/21 01:09:29 UTC

Change in asterixdb[neo]: [NO ISSUE][HYR] Fix switching search on memory component to disk comp...

From Ali Alsuliman <al...@gmail.com>:

Ali Alsuliman has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/14885 )


Change subject: [NO ISSUE][HYR] Fix switching search on memory component to disk component
......................................................................

[NO ISSUE][HYR] Fix switching search on memory component to disk component

- user model changes: no
- storage format changes: no
- interface changes: no

Details:

Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
---
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
3 files changed, 33 insertions(+), 21 deletions(-)



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

diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 46d279f..bbbc561 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
@@ -229,14 +230,14 @@
                 if (i == 0 && component.getType() != LSMComponentType.MEMORY) {
                     includeMutableComponent = false;
                 }
-                if (switchedElements[i] != null) {
-                    copyTuple.reset(switchComponentTupleBuilders[i].getFieldEndOffsets(),
-                            switchComponentTupleBuilders[i].getByteArray());
-                    reusablePred.setLowKey(copyTuple, true);
-                    rangeCursors[i].close();
-                    btreeAccessors[i].reset(btree, iap);
-                    btreeAccessors[i].search(rangeCursors[i], reusablePred);
-                    pushIntoQueueFromCursorAndReplaceThisElement(switchedElements[i]);
+                copyTuple.reset(switchComponentTupleBuilders[i].getFieldEndOffsets(),
+                        switchComponentTupleBuilders[i].getByteArray());
+                reusablePred.setLowKey(copyTuple, true);
+                rangeCursors[i].close();
+                btreeAccessors[i].reset(btree, iap);
+                btreeAccessors[i].search(rangeCursors[i], reusablePred);
+                if (rangeCursors[i].hasNext()) {
+                    rangeCursors[i].next();
                 }
             }
             switchRequest[i] = false;
@@ -265,25 +266,26 @@
                     replaceFrom = i;
                 }
                 // we return the outputElement to the priority queue if it came from this component
-                if (outputElement != null && outputElement.getCursorIndex() == i) {
-                    pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
-                    needPushElementIntoQueue = false;
-                    outputElement = null;
-                    canCallProceed = true;
+                ITupleReference tuple = null;
+                PriorityQueueElement componentElement = findElement(outputPriorityQueue, i);
+                if (componentElement != null) {
+                    tuple = componentElement.getTuple();
+                } else if (outputElement != null && outputElement.getCursorIndex() == i) {
+                    tuple = outputElement.getTuple();
                 }
-                PriorityQueueElement element = remove(outputPriorityQueue, i);
                 // if this cursor is still active (has an element)
                 // then we copy the search key to restart the operation after
                 // replacing the component
-                if (element != null) {
+                if (tuple != null) {
                     if (switchComponentTupleBuilders[i] == null) {
                         switchComponentTupleBuilders[i] = new ArrayTupleBuilder(cmp.getKeyFieldCount());
                     }
-                    TupleUtils.copyTuple(switchComponentTupleBuilders[i], element.getTuple(), cmp.getKeyFieldCount());
+                    TupleUtils.copyTuple(switchComponentTupleBuilders[i], tuple, cmp.getKeyFieldCount());
+                    switchRequest[i] = true;
+                } else {
+                    switchRequest[i] = false;
                 }
                 rangeCursors[i].close();
-                switchRequest[i] = true;
-                switchedElements[i] = element;
             }
         }
         return replaceFrom;
@@ -331,6 +333,18 @@
         return null;
     }
 
+    private PriorityQueueElement findElement(PriorityQueue<PriorityQueueElement> outputPriorityQueue, int cursorIndex) {
+        // Scans the PQ for the component's element
+        Iterator<PriorityQueueElement> it = outputPriorityQueue.iterator();
+        while (it.hasNext()) {
+            PriorityQueueElement e = it.next();
+            if (e.getCursorIndex() == cursorIndex) {
+                return e;
+            }
+        }
+        return null;
+    }
+
     @Override
     public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
index 70f6d9e..6a99da1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
@@ -193,7 +193,7 @@
             for (int i = 0; i < count; i++) {
                 ILSMComponent removed = ctx.getComponentHolder().remove(swapIndexes[i]);
                 if (removed.getType() == LSMComponentType.MEMORY) {
-                    LOGGER.info("Removed a memory component from the search operation");
+                    LOGGER.info("Removed memory component {} from the search operation", removed);
                 } else {
                     throw new IllegalStateException("Disk components can't be removed from the search operation");
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index b6f6e26..e46bc47 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -46,7 +46,6 @@
     protected PriorityQueueElement outputElement;
     protected final ArrayTupleBuilder[] switchComponentTupleBuilders;
     protected final boolean[] switchRequest;
-    protected final PriorityQueueElement[] switchedElements;
     protected IIndexCursor[] rangeCursors;
     protected PriorityQueueElement[] pqes;
     protected PriorityQueue<PriorityQueueElement> outputPriorityQueue;
@@ -68,7 +67,6 @@
         needPushElementIntoQueue = false;
         switchComponentTupleBuilders = new ArrayTupleBuilder[opCtx.getIndex().getNumberOfAllMemoryComponents()];
         switchRequest = new boolean[switchComponentTupleBuilders.length];
-        switchedElements = new PriorityQueueElement[switchComponentTupleBuilders.length];
         this.iap = IndexAccessParameters.createNoOpParams(stats);
     }
 

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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-MessageType: newchange

Change in asterixdb[neo]: [NO ISSUE][STO] Fix switching search on memory component to disk comp...

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/+/14885 )

Change subject: [NO ISSUE][STO] Fix switching search on memory component to disk component
......................................................................


Patch Set 3:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 3
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Comment-Date: Mon, 24 Jan 2022 22:46:22 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[neo]: [NO ISSUE][HYR] Fix switching search on memory component to disk comp...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ali Alsuliman <al...@gmail.com>:

Ali Alsuliman has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/14885 )


Change subject: [NO ISSUE][HYR] Fix switching search on memory component to disk component
......................................................................

[NO ISSUE][HYR] Fix switching search on memory component to disk component

- user model changes: no
- storage format changes: no
- interface changes: no

Details:

Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
---
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
3 files changed, 33 insertions(+), 21 deletions(-)



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

diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
index 46d279f..bbbc561 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
@@ -26,6 +26,7 @@
 import org.apache.hyracks.api.util.CleanupUtils;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleBuilder;
 import org.apache.hyracks.dataflow.common.comm.io.ArrayTupleReference;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
 import org.apache.hyracks.dataflow.common.utils.TupleUtils;
 import org.apache.hyracks.storage.am.btree.impls.BTree;
 import org.apache.hyracks.storage.am.btree.impls.BTree.BTreeAccessor;
@@ -229,14 +230,14 @@
                 if (i == 0 && component.getType() != LSMComponentType.MEMORY) {
                     includeMutableComponent = false;
                 }
-                if (switchedElements[i] != null) {
-                    copyTuple.reset(switchComponentTupleBuilders[i].getFieldEndOffsets(),
-                            switchComponentTupleBuilders[i].getByteArray());
-                    reusablePred.setLowKey(copyTuple, true);
-                    rangeCursors[i].close();
-                    btreeAccessors[i].reset(btree, iap);
-                    btreeAccessors[i].search(rangeCursors[i], reusablePred);
-                    pushIntoQueueFromCursorAndReplaceThisElement(switchedElements[i]);
+                copyTuple.reset(switchComponentTupleBuilders[i].getFieldEndOffsets(),
+                        switchComponentTupleBuilders[i].getByteArray());
+                reusablePred.setLowKey(copyTuple, true);
+                rangeCursors[i].close();
+                btreeAccessors[i].reset(btree, iap);
+                btreeAccessors[i].search(rangeCursors[i], reusablePred);
+                if (rangeCursors[i].hasNext()) {
+                    rangeCursors[i].next();
                 }
             }
             switchRequest[i] = false;
@@ -265,25 +266,26 @@
                     replaceFrom = i;
                 }
                 // we return the outputElement to the priority queue if it came from this component
-                if (outputElement != null && outputElement.getCursorIndex() == i) {
-                    pushIntoQueueFromCursorAndReplaceThisElement(outputElement);
-                    needPushElementIntoQueue = false;
-                    outputElement = null;
-                    canCallProceed = true;
+                ITupleReference tuple = null;
+                PriorityQueueElement componentElement = findElement(outputPriorityQueue, i);
+                if (componentElement != null) {
+                    tuple = componentElement.getTuple();
+                } else if (outputElement != null && outputElement.getCursorIndex() == i) {
+                    tuple = outputElement.getTuple();
                 }
-                PriorityQueueElement element = remove(outputPriorityQueue, i);
                 // if this cursor is still active (has an element)
                 // then we copy the search key to restart the operation after
                 // replacing the component
-                if (element != null) {
+                if (tuple != null) {
                     if (switchComponentTupleBuilders[i] == null) {
                         switchComponentTupleBuilders[i] = new ArrayTupleBuilder(cmp.getKeyFieldCount());
                     }
-                    TupleUtils.copyTuple(switchComponentTupleBuilders[i], element.getTuple(), cmp.getKeyFieldCount());
+                    TupleUtils.copyTuple(switchComponentTupleBuilders[i], tuple, cmp.getKeyFieldCount());
+                    switchRequest[i] = true;
+                } else {
+                    switchRequest[i] = false;
                 }
                 rangeCursors[i].close();
-                switchRequest[i] = true;
-                switchedElements[i] = element;
             }
         }
         return replaceFrom;
@@ -331,6 +333,18 @@
         return null;
     }
 
+    private PriorityQueueElement findElement(PriorityQueue<PriorityQueueElement> outputPriorityQueue, int cursorIndex) {
+        // Scans the PQ for the component's element
+        Iterator<PriorityQueueElement> it = outputPriorityQueue.iterator();
+        while (it.hasNext()) {
+            PriorityQueueElement e = it.next();
+            if (e.getCursorIndex() == cursorIndex) {
+                return e;
+            }
+        }
+        return null;
+    }
+
     @Override
     public void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
         LSMBTreeCursorInitialState lsmInitialState = (LSMBTreeCursorInitialState) initialState;
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
index 70f6d9e..6a99da1 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
@@ -193,7 +193,7 @@
             for (int i = 0; i < count; i++) {
                 ILSMComponent removed = ctx.getComponentHolder().remove(swapIndexes[i]);
                 if (removed.getType() == LSMComponentType.MEMORY) {
-                    LOGGER.info("Removed a memory component from the search operation");
+                    LOGGER.info("Removed memory component {} from the search operation", removed);
                 } else {
                     throw new IllegalStateException("Disk components can't be removed from the search operation");
                 }
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
index b6f6e26..e46bc47 100644
--- a/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
+++ b/hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
@@ -46,7 +46,6 @@
     protected PriorityQueueElement outputElement;
     protected final ArrayTupleBuilder[] switchComponentTupleBuilders;
     protected final boolean[] switchRequest;
-    protected final PriorityQueueElement[] switchedElements;
     protected IIndexCursor[] rangeCursors;
     protected PriorityQueueElement[] pqes;
     protected PriorityQueue<PriorityQueueElement> outputPriorityQueue;
@@ -68,7 +67,6 @@
         needPushElementIntoQueue = false;
         switchComponentTupleBuilders = new ArrayTupleBuilder[opCtx.getIndex().getNumberOfAllMemoryComponents()];
         switchRequest = new boolean[switchComponentTupleBuilders.length];
-        switchedElements = new PriorityQueueElement[switchComponentTupleBuilders.length];
         this.iap = IndexAccessParameters.createNoOpParams(stats);
     }
 

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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-MessageType: newchange

Change in asterixdb[neo]: [NO ISSUE][HYR] Fix switching search on memory component to disk comp...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

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

Change subject: [NO ISSUE][HYR] Fix switching search on memory component to disk component
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sun, 23 Jan 2022 21:42:03 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[neo]: [NO ISSUE][HYR] Fix switching search on memory component to disk comp...

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

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

Change subject: [NO ISSUE][HYR] Fix switching search on memory component to disk component
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 1
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-CC: Anon. E. Moose #1000171
Gerrit-Comment-Date: Fri, 21 Jan 2022 02:11:36 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[neo]: [NO ISSUE][STO] Fix switching cursor from memory to disk component

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Jenkins <je...@fulliautomatix.ics.uci.edu>:

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

Change subject: [NO ISSUE][STO] Fix switching cursor from memory to disk component
......................................................................


Patch Set 4: Integration-Tests+1

Integration Tests Successful

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 4
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Comment-Date: Mon, 24 Jan 2022 23:49:20 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[neo]: [NO ISSUE][STO] Fix search when switching from memory to disk component

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/+/14885 )

Change subject: [NO ISSUE][STO] Fix search when switching from memory to disk component
......................................................................


Patch Set 6: Contrib-2

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 6
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Comment-Date: Tue, 25 Jan 2022 08:42:07 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[neo]: [NO ISSUE][STO] Fix search when switching from memory to disk component

Posted by AsterixDB Code Review <do...@asterix-gerrit.ics.uci.edu>.
From Ali Alsuliman <al...@gmail.com>:

Hello Murtadha Hubail, Jenkins, Anon. E. Moose #1000171, 

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

    https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/14885

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

Change subject: [NO ISSUE][STO] Fix search when switching from memory to disk component
......................................................................

[NO ISSUE][STO] Fix search when switching from memory to disk component

- user model changes: no
- storage format changes: no
- interface changes: no

Details:
- When searching the index and making the switch from the memory
  components to the disk components, keep the states of the queue and
  the cursors on the switched-to disk components the same as their
  states were on the memory components. If a cursor was the one who
  produced the outputElement, then do not push the next element into
  the queue from the cursor since there should not be an element in
  the queue from this cursor. Restart the search operation at the
  elements that the cursors were at and consume them since they were
  already consumed before we make the switch.

- add test case.

Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
---
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/bootstrap/TestNodeController.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/SearchCursorComponentSwitchTest.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-btree/src/main/java/org/apache/hyracks/storage/am/lsm/btree/impls/LSMBTreeRangeSearchCursor.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/ComponentReplacementContext.java
M hyracks-fullstack/hyracks/hyracks-storage-am-lsm-common/src/main/java/org/apache/hyracks/storage/am/lsm/common/impls/LSMIndexSearchCursor.java
5 files changed, 138 insertions(+), 25 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/85/14885/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/c/asterixdb/+/14885
To unsubscribe, or for help writing mail filters, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 6
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-MessageType: newpatchset

Change in asterixdb[neo]: [NO ISSUE][HYR] Fix switching search on memory component to disk comp...

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/+/14885 )

Change subject: [NO ISSUE][HYR] Fix switching search on memory component to disk component
......................................................................


Patch Set 2:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sun, 23 Jan 2022 20:45:53 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment

Change in asterixdb[neo]: [NO ISSUE][HYR] Fix switching search on memory component to disk comp...

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/+/14885 )

Change subject: [NO ISSUE][HYR] Fix switching search on memory component to disk component
......................................................................


Patch Set 2: Contrib-2

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 2
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Sun, 23 Jan 2022 23:02:40 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: Yes
Gerrit-MessageType: comment

Change in asterixdb[neo]: [NO ISSUE][STO] Fix search when switching from memory to disk component

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/+/14885 )

Change subject: [NO ISSUE][STO] Fix search when switching from memory to disk component
......................................................................


Patch Set 6:

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


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

Gerrit-Project: asterixdb
Gerrit-Branch: neo
Gerrit-Change-Id: I647641f6044c1edf1477049be1c5d1b697f404c1
Gerrit-Change-Number: 14885
Gerrit-PatchSet: 6
Gerrit-Owner: Ali Alsuliman <al...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Comment-Date: Tue, 25 Jan 2022 06:33:51 +0000
Gerrit-HasComments: No
Gerrit-Has-Labels: No
Gerrit-MessageType: comment