You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/01/17 01:59:00 UTC

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

abdullah alamoudi has uploaded a new change for review.

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................

[ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor

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

details:
- Add a base test for all implementations of IIndexCursor
- The test case includes the following scenarios:
--- testNormalLifeCycle
--- testCreateDestroySucceed
--- testDoubleOpenFails
--- testCloseWithoutOpenFails
--- testDoubleCloseFails
--- testHasNextBeforeOpenFails
--- testHasNextAfterCloseFails
--- testNextBeforeOpenFails
--- testNextAfterCloseFails
--- testDestroyWhileOpenFails
--- testOpenAfterDestroyFails
--- testCloseAfterDestroyFails
--- testNextAfterDestroyFails
--- testHasNextAfterDestroyFails
--- testGetTupleReturnsNullAfterDestroy
- Add a base implementation of the interface that conforms
  to the expected lifecycle and can be extended by
  any cursor implementation.
- The test is run on the base implementation.

Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
---
A hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
A hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
A hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
3 files changed, 423 insertions(+), 0 deletions(-)


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

diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
new file mode 100644
index 0000000..8fe689e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+package org.apache.hyracks.tests.unit;
+
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class EnforcedIndexCursorTest extends IIndexCursorTest {
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        for (int i = 0; i < 10; i++) {
+            predicates.add(Mockito.mock(ISearchPredicate.class));
+        }
+        return predicates;
+    }
+
+    @Override
+    protected ICursorInitialState createCursorInitialState() {
+        return Mockito.mock(ICursorInitialState.class);
+    }
+
+    @Override
+    protected IIndexCursor createCursor() {
+        return new EnforcedIndexCursor();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
new file mode 100644
index 0000000..4d0f287
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
@@ -0,0 +1,263 @@
+/*
+ * 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.
+ */
+
+package org.apache.hyracks.tests.unit;
+
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * This is a test class that forms the basis for unit tests of different implementations of the IIndexCursor interface
+ */
+public abstract class IIndexCursorTest {
+    @Test
+    public void testNormalLifeCycle() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        for (ISearchPredicate predicate : predicates) {
+            cursor.open(initialState, predicate);
+            while (cursor.hasNext()) {
+                cursor.next();
+            }
+            cursor.close();
+        }
+        cursor.destroy();
+    }
+
+    @Test
+    public void testCreateDestroySucceed() throws Exception {
+        IIndexCursor cursor = createCursor();
+        cursor.destroy();
+    }
+
+    @Test
+    public void testDoubleOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.open(initialState, predicates.get(0));
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.close();
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testCloseWithoutOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testDoubleCloseFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testHasNextBeforeOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testHasNextAfterCloseFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextBeforeOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextAfterCloseFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testDestroyWhileOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.destroy();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.close();
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testOpenAfterDestroyFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.open(initialState, predicates.get(0));
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testCloseAfterDestroyFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextAfterDestroyFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testHasNextAfterDestroyFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testGetTupleReturnsNullAfterDestroy() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        Assert.assertNull(cursor.getTuple());
+    }
+
+    protected abstract List<ISearchPredicate> createSearchPredicates();
+
+    protected abstract ICursorInitialState createCursorInitialState();
+
+    protected abstract IIndexCursor createCursor();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
new file mode 100644
index 0000000..62a56e6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
@@ -0,0 +1,111 @@
+/*
+ * 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.
+ */
+
+package org.apache.hyracks.storage.common;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class EnforcedIndexCursor implements IIndexCursor {
+    enum State {
+        CLOSED,
+        OPENED,
+        DESTROYED
+    }
+
+    private State state = State.CLOSED;
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        if (state != State.CLOSED) {
+            throw new IllegalStateException("Cannot open a cursor in the state " + state);
+        }
+        doOpen(initialState, searchPred);
+        state = State.OPENED;
+    }
+
+    protected void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException {
+        if (state != State.OPENED) {
+            throw new IllegalStateException("Cannot call hasNext() on a cursor in the state " + state);
+        }
+        return doHasNext();
+    }
+
+    protected boolean doHasNext() throws HyracksDataException {
+        return false;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        if (state != State.OPENED) {
+            throw new IllegalStateException("Cannot call next() on a cursor in the state " + state);
+        }
+        doNext();
+    }
+
+    protected void doNext() throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (state != State.CLOSED) {
+            throw new IllegalStateException("Cannot destroy a cursor in the state " + state);
+        }
+        doDestroy();
+        state = State.DESTROYED;
+    }
+
+    protected void doDestroy() throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (state != State.OPENED) {
+            throw new IllegalStateException("Cannot close a cursor in the state " + state);
+        }
+        doClose();
+        state = State.CLOSED;
+    }
+
+    private void doClose() throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return null;
+    }
+
+    @Override
+    public ITupleReference getFilterMinTuple() {
+        return null;
+    }
+
+    @Override
+    public ITupleReference getFilterMaxTuple() {
+        return null;
+    }
+}

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-rebalance-cancellation/1475/ (4/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/6283/ (2/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/2883/ (10/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/2278/ (1/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/2563/ (9/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/2256/ (8/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2491/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
abdullah alamoudi has submitted this change and it was merged.

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


[ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor

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

details:
- Add a base test for all implementations of IIndexCursor
- The test case includes the following scenarios:
--- testNormalLifeCycle
--- testCreateDestroySucceed
--- testDoubleOpenFails
--- testCloseWithoutOpenFails
--- testDoubleCloseFails
--- testHasNextBeforeOpenFails
--- testHasNextAfterCloseFails
--- testNextBeforeOpenFails
--- testNextAfterCloseFails
--- testDestroyWhileOpenFails
--- testOpenAfterDestroyFails
--- testCloseAfterDestroyFails
--- testNextAfterDestroyFails
--- testHasNextAfterDestroyFails
--- testGetTupleReturnsNullAfterDestroy
- Add a base implementation of the interface that conforms
  to the expected lifecycle and can be extended by
  any cursor implementation.
- The test is run on the base implementation.

Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2291
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mb...@apache.org>
---
A hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
A hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
A hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
3 files changed, 423 insertions(+), 0 deletions(-)

Approvals:
  Anon. E. Moose #1000171: 
  Jenkins: Verified; No violations found; ; Verified
  Michael Blow: Looks good to me, approved



diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
new file mode 100644
index 0000000..8fe689e
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/EnforcedIndexCursorTest.java
@@ -0,0 +1,49 @@
+/*
+ * 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.
+ */
+package org.apache.hyracks.tests.unit;
+
+import org.apache.hyracks.storage.common.EnforcedIndexCursor;
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class EnforcedIndexCursorTest extends IIndexCursorTest {
+    @Override
+    protected List<ISearchPredicate> createSearchPredicates() {
+        List<ISearchPredicate> predicates = new ArrayList<>();
+        for (int i = 0; i < 10; i++) {
+            predicates.add(Mockito.mock(ISearchPredicate.class));
+        }
+        return predicates;
+    }
+
+    @Override
+    protected ICursorInitialState createCursorInitialState() {
+        return Mockito.mock(ICursorInitialState.class);
+    }
+
+    @Override
+    protected IIndexCursor createCursor() {
+        return new EnforcedIndexCursor();
+    }
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
new file mode 100644
index 0000000..4d0f287
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-examples/hyracks-integration-tests/src/test/java/org/apache/hyracks/tests/unit/IIndexCursorTest.java
@@ -0,0 +1,263 @@
+/*
+ * 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.
+ */
+
+package org.apache.hyracks.tests.unit;
+
+import org.apache.hyracks.storage.common.ICursorInitialState;
+import org.apache.hyracks.storage.common.IIndexCursor;
+import org.apache.hyracks.storage.common.ISearchPredicate;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.util.List;
+
+/**
+ * This is a test class that forms the basis for unit tests of different implementations of the IIndexCursor interface
+ */
+public abstract class IIndexCursorTest {
+    @Test
+    public void testNormalLifeCycle() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        for (ISearchPredicate predicate : predicates) {
+            cursor.open(initialState, predicate);
+            while (cursor.hasNext()) {
+                cursor.next();
+            }
+            cursor.close();
+        }
+        cursor.destroy();
+    }
+
+    @Test
+    public void testCreateDestroySucceed() throws Exception {
+        IIndexCursor cursor = createCursor();
+        cursor.destroy();
+    }
+
+    @Test
+    public void testDoubleOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.open(initialState, predicates.get(0));
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.close();
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testCloseWithoutOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testDoubleCloseFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testHasNextBeforeOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testHasNextAfterCloseFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextBeforeOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextAfterCloseFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testDestroyWhileOpenFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.destroy();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        cursor.close();
+        cursor.destroy();
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testOpenAfterDestroyFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.open(initialState, predicates.get(0));
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testCloseAfterDestroyFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.close();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testNextAfterDestroyFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.next();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testHasNextAfterDestroyFails() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        boolean expectedExceptionThrown = false;
+        try {
+            cursor.hasNext();
+        } catch (Exception e) {
+            expectedExceptionThrown = true;
+        }
+        Assert.assertTrue(expectedExceptionThrown);
+    }
+
+    @Test
+    public void testGetTupleReturnsNullAfterDestroy() throws Exception {
+        IIndexCursor cursor = createCursor();
+        ICursorInitialState initialState = createCursorInitialState();
+        List<ISearchPredicate> predicates = createSearchPredicates();
+        cursor.open(initialState, predicates.get(0));
+        cursor.close();
+        cursor.destroy();
+        Assert.assertNull(cursor.getTuple());
+    }
+
+    protected abstract List<ISearchPredicate> createSearchPredicates();
+
+    protected abstract ICursorInitialState createCursorInitialState();
+
+    protected abstract IIndexCursor createCursor();
+}
diff --git a/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
new file mode 100644
index 0000000..62a56e6
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-storage-common/src/main/java/org/apache/hyracks/storage/common/EnforcedIndexCursor.java
@@ -0,0 +1,111 @@
+/*
+ * 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.
+ */
+
+package org.apache.hyracks.storage.common;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public class EnforcedIndexCursor implements IIndexCursor {
+    enum State {
+        CLOSED,
+        OPENED,
+        DESTROYED
+    }
+
+    private State state = State.CLOSED;
+
+    @Override
+    public void open(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        if (state != State.CLOSED) {
+            throw new IllegalStateException("Cannot open a cursor in the state " + state);
+        }
+        doOpen(initialState, searchPred);
+        state = State.OPENED;
+    }
+
+    protected void doOpen(ICursorInitialState initialState, ISearchPredicate searchPred) throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public boolean hasNext() throws HyracksDataException {
+        if (state != State.OPENED) {
+            throw new IllegalStateException("Cannot call hasNext() on a cursor in the state " + state);
+        }
+        return doHasNext();
+    }
+
+    protected boolean doHasNext() throws HyracksDataException {
+        return false;
+    }
+
+    @Override
+    public void next() throws HyracksDataException {
+        if (state != State.OPENED) {
+            throw new IllegalStateException("Cannot call next() on a cursor in the state " + state);
+        }
+        doNext();
+    }
+
+    protected void doNext() throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public void destroy() throws HyracksDataException {
+        if (state != State.CLOSED) {
+            throw new IllegalStateException("Cannot destroy a cursor in the state " + state);
+        }
+        doDestroy();
+        state = State.DESTROYED;
+    }
+
+    protected void doDestroy() throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        if (state != State.OPENED) {
+            throw new IllegalStateException("Cannot close a cursor in the state " + state);
+        }
+        doClose();
+        state = State.CLOSED;
+    }
+
+    private void doClose() throws HyracksDataException {
+        // Do nothing
+    }
+
+    @Override
+    public ITupleReference getTuple() {
+        return null;
+    }
+
+    @Override
+    public ITupleReference getFilterMinTuple() {
+        return null;
+    }
+
+    @Override
+    public ITupleReference getFilterMaxTuple() {
+        return null;
+    }
+}

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/2864/ (5/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/3109/ (12/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/7788/ (7/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/2234/ (6/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://goo.gl/aF6YEZ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/GY3Te8 : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/2863/ (11/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/2491/ : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2204][STO] Add an abstract test case for an IInde...

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

Change subject: [ASTERIXDB-2204][STO] Add an abstract test case for an IIndexCursor
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/341/ (3/12)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I7c32dd560367d84403ffa3d9cb69ff80d715fdc5
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No