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 2017/08/27 03:48:51 UTC

Change in asterixdb[master]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

abdullah alamoudi has uploaded a new change for review.

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................

[NO ISSUE][*DB] Prevent Cancellation for DMLs

- user model changes: The only cancellable jobs are queries.
- storage format changes: no
- interface changes: no

details:
- Before this change, DML jobs were added to the job execution
  context.
- This job passes a dummy job execution context to disallow
  registration and cancellation of DMLs.

Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
---
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
2 files changed, 49 insertions(+), 4 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
new file mode 100644
index 0000000..ec181bb
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
@@ -0,0 +1,45 @@
+/*
+ * 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.asterix.translator;
+
+import org.apache.hyracks.api.job.JobId;
+
+public class NoOpStatementExecutorContext implements IStatementExecutorContext {
+
+    public static final NoOpStatementExecutorContext INSTANCE = new NoOpStatementExecutorContext();
+
+    private NoOpStatementExecutorContext() {
+    }
+
+    @Override
+    public JobId getJobIdFromClientContextId(String clientContextId) {
+        return null;
+    }
+
+    @Override
+    public void put(String clientContextId, JobId jobId) {
+        // Dummy for when a statement doesn't support cancellation
+    }
+
+    @Override
+    public JobId removeJobIdFromClientContextId(String clientContextId) {
+        return null;
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index b97c014..e0d56a0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -162,6 +162,7 @@
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutorContext;
+import org.apache.asterix.translator.NoOpStatementExecutorContext;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
 import org.apache.asterix.translator.TypeTranslator;
@@ -334,7 +335,7 @@
                                     || resultDelivery == ResultDelivery.DEFERRED);
                         }
                         handleInsertUpsertStatement(metadataProvider, stmt, hcc, hdc, resultDelivery, outMetadata,
-                                stats, false, clientContextId, ctx);
+                                stats, false, clientContextId);
                         break;
                     case Statement.Kind.DELETE:
                         handleDeleteStatement(metadataProvider, stmt, hcc, false);
@@ -1727,8 +1728,7 @@
 
     public JobSpecification handleInsertUpsertStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc, IHyracksDataset hdc, ResultDelivery resultDelivery,
-            ResultMetadata outMetadata, Stats stats, boolean compileOnly, String clientContextId,
-            IStatementExecutorContext ctx) throws Exception {
+            ResultMetadata outMetadata, Stats stats, boolean compileOnly, String clientContextId) throws Exception {
         InsertStatement stmtInsertUpsert = (InsertStatement) stmt;
         String dataverseName = getActiveDataverse(stmtInsertUpsert.getDataverseName());
         final IMetadataLocker locker = new IMetadataLocker() {
@@ -1771,7 +1771,7 @@
 
         if (stmtInsertUpsert.getReturnExpression() != null) {
             deliverResult(hcc, hdc, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats,
-                    clientContextId, ctx);
+                    clientContextId, NoOpStatementExecutorContext.INSTANCE);
         } else {
             locker.lock();
             try {

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

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

Change in asterixdb[master]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1: Contrib-1

Analytics Compatibility Tests Failed Build 2516
: UNSTABLE

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

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

Change in asterixdb[master]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/779/ (9/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1: Contrib-1

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/1710/ : FAILURE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/1702/ (8/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/784/ (3/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-rebalance-cancellation/607/ (6/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/1399/ (1/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


[NO ISSUE][*DB] Prevent Cancellation for DMLs

- user model changes: The only cancellable jobs are queries.
- storage format changes: no
- interface changes: no

details:
- Before this change, DML jobs were added to the job execution
  context.
- This job passes a dummy job execution context to disallow
  registration and cancellation of DMLs.

Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1977
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mb...@apache.org>
---
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
2 files changed, 49 insertions(+), 4 deletions(-)

Approvals:
  Jenkins: Verified; No violations found; Verified
  Michael Blow: Looks good to me, approved

Objections:
  Anon. E. Moose #1000171: 
  Jenkins: 



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
new file mode 100644
index 0000000..ec181bb
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
@@ -0,0 +1,45 @@
+/*
+ * 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.asterix.translator;
+
+import org.apache.hyracks.api.job.JobId;
+
+public class NoOpStatementExecutorContext implements IStatementExecutorContext {
+
+    public static final NoOpStatementExecutorContext INSTANCE = new NoOpStatementExecutorContext();
+
+    private NoOpStatementExecutorContext() {
+    }
+
+    @Override
+    public JobId getJobIdFromClientContextId(String clientContextId) {
+        return null;
+    }
+
+    @Override
+    public void put(String clientContextId, JobId jobId) {
+        // Dummy for when a statement doesn't support cancellation
+    }
+
+    @Override
+    public JobId removeJobIdFromClientContextId(String clientContextId) {
+        return null;
+    }
+
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index b97c014..e0d56a0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -162,6 +162,7 @@
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutorContext;
+import org.apache.asterix.translator.NoOpStatementExecutorContext;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
 import org.apache.asterix.translator.TypeTranslator;
@@ -334,7 +335,7 @@
                                     || resultDelivery == ResultDelivery.DEFERRED);
                         }
                         handleInsertUpsertStatement(metadataProvider, stmt, hcc, hdc, resultDelivery, outMetadata,
-                                stats, false, clientContextId, ctx);
+                                stats, false, clientContextId);
                         break;
                     case Statement.Kind.DELETE:
                         handleDeleteStatement(metadataProvider, stmt, hcc, false);
@@ -1727,8 +1728,7 @@
 
     public JobSpecification handleInsertUpsertStatement(MetadataProvider metadataProvider, Statement stmt,
             IHyracksClientConnection hcc, IHyracksDataset hdc, ResultDelivery resultDelivery,
-            ResultMetadata outMetadata, Stats stats, boolean compileOnly, String clientContextId,
-            IStatementExecutorContext ctx) throws Exception {
+            ResultMetadata outMetadata, Stats stats, boolean compileOnly, String clientContextId) throws Exception {
         InsertStatement stmtInsertUpsert = (InsertStatement) stmt;
         String dataverseName = getActiveDataverse(stmtInsertUpsert.getDataverseName());
         final IMetadataLocker locker = new IMetadataLocker() {
@@ -1771,7 +1771,7 @@
 
         if (stmtInsertUpsert.getReturnExpression() != null) {
             deliverResult(hcc, hdc, compiler, metadataProvider, locker, resultDelivery, outMetadata, stats,
-                    clientContextId, ctx);
+                    clientContextId, NoOpStatementExecutorContext.INSTANCE);
         } else {
             locker.lock();
             try {

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: abdullah alamoudi <ba...@gmail.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Michael Blow <mb...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/786/ (11/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/6338/ (5/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/1122/ (11/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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]: [NO ISSUE][*DB] Prevent Cancellation for DMLs

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

Change subject: [NO ISSUE][*DB] Prevent Cancellation for DMLs
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/785/ (4/11)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I3bb1a737c6ec6dd5a79fc25db25325ab60d838b9
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