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/07/17 04:45:54 UTC

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

abdullah alamoudi has uploaded a new change for review.

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................

[NO ISSUE][MISC] Introduce IClientRequest

- user model changes: no
- storage format changes: no
- interface changes: yes
  + IClientRequest: used to represent a client request
    that can be cancelled.

Details:
- Introduce IClientRequest to allow for multiple types of requests
  to be cancellable.

Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
R asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IStatementExecutorContext.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseClientRequest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ClientJobRequest.java
15 files changed, 202 insertions(+), 64 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index fa967f5..2e644a9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -24,6 +24,7 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.api.IStatementExecutorContext;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
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
index ec181bb..4c65fec 100644
--- 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
@@ -18,7 +18,8 @@
  */
 package org.apache.asterix.translator;
 
-import org.apache.hyracks.api.job.JobId;
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.api.IStatementExecutorContext;
 
 public class NoOpStatementExecutorContext implements IStatementExecutorContext {
 
@@ -28,17 +29,17 @@
     }
 
     @Override
-    public JobId getJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest get(String clientContextId) {
         return null;
     }
 
     @Override
-    public void put(String clientContextId, JobId jobId) {
+    public void put(String clientContextId, IClientRequest req) {
         // Dummy for when a statement doesn't support cancellation
     }
 
     @Override
-    public JobId removeJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest remove(String clientContextId) {
         return null;
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
index 7c06762..1af975c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
@@ -16,31 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.api.http.ctx;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.job.JobId;
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.api.IStatementExecutorContext;
 
 public class StatementExecutorContext implements IStatementExecutorContext {
 
-    private final Map<String, JobId> runningQueries = new ConcurrentHashMap<>();
+    private final Map<String, IClientRequest> runningQueries = new ConcurrentHashMap<>();
 
     @Override
-    public JobId getJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest get(String clientContextId) {
         return runningQueries.get(clientContextId);
     }
 
     @Override
-    public void put(String clientContextId, JobId jobId) {
-        runningQueries.put(clientContextId, jobId);
+    public void put(String clientContextId, IClientRequest req) {
+        runningQueries.put(clientContextId, req);
     }
 
     @Override
-    public JobId removeJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest remove(String clientContextId) {
         return runningQueries.remove(clientContextId);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
similarity index 74%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
index a8b3aef..4438f2e 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
@@ -22,9 +22,9 @@
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.api.http.server.QueryServiceServlet.Parameter;
-import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.job.JobId;
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.api.IStatementExecutorContext;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.AbstractServlet;
@@ -37,11 +37,14 @@
 /**
  * The servlet provides a REST API for cancelling an on-going query.
  */
-public class QueryCancellationServlet extends AbstractServlet {
+public class CcQueryCancellationServlet extends AbstractServlet {
     private static final Logger LOGGER = LogManager.getLogger();
+    private final ICcApplicationContext appCtx;
 
-    public QueryCancellationServlet(ConcurrentMap<String, Object> ctx, String... paths) {
+    public CcQueryCancellationServlet(ConcurrentMap<String, Object> ctx, ICcApplicationContext appCtx,
+            String... paths) {
         super(ctx, paths);
+        this.appCtx = appCtx;
     }
 
     @Override
@@ -51,23 +54,17 @@
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
         }
-
-        // Retrieves the corresponding Hyracks job id.
-        IStatementExecutorContext runningQueries =
+        IStatementExecutorContext executorCtx =
                 (IStatementExecutorContext) ctx.get(ServletConstants.RUNNING_QUERIES_ATTR);
-        IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(ServletConstants.HYRACKS_CONNECTION_ATTR);
-        JobId jobId = runningQueries.getJobIdFromClientContextId(clientContextId);
-
-        if (jobId == null) {
+        IClientRequest req = executorCtx.get(clientContextId);
+        if (req == null) {
             // response: NOT FOUND
             response.setStatus(HttpResponseStatus.NOT_FOUND);
             return;
         }
         try {
             // Cancels the on-going job.
-            hcc.cancelJob(jobId);
-            // Removes the cancelled query from the map activeQueries.
-            runningQueries.removeJobIdFromClientContextId(clientContextId);
+            req.cancel(appCtx);
             // response: OK
             response.setStatus(HttpResponseStatus.OK);
         } catch (Exception e) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
index 5ad451f..60806d3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
+import org.apache.hyracks.http.server.AbstractServlet;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -41,7 +42,7 @@
 /**
  * The servlet provides a REST API on an NC for cancelling an on-going query.
  */
-public class NCQueryCancellationServlet extends QueryCancellationServlet {
+public class NCQueryCancellationServlet extends AbstractServlet {
     private static final Logger LOGGER = LogManager.getLogger();
     private final INCServiceContext serviceCtx;
     private final INCMessageBroker messageBroker;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index ce6185e..2311a5d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -39,6 +39,7 @@
 import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.api.IStatementExecutorContext;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
@@ -55,7 +56,6 @@
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutor.ResultDelivery;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
-import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.ResultProperties;
 import org.apache.asterix.translator.SessionConfig;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
index a1c87d8..38813f0 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
@@ -18,15 +18,14 @@
  */
 package org.apache.asterix.app.message;
 
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.api.IStatementExecutorContext;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
 import org.apache.asterix.common.utils.RequestStatus;
 import org.apache.asterix.hyracks.bootstrap.CCApplication;
 import org.apache.asterix.messaging.CCMessageBroker;
-import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -51,17 +50,16 @@
         ClusterControllerService ccs = (ClusterControllerService) appCtx.getServiceContext().getControllerService();
         CCApplication application = (CCApplication) ccs.getApplication();
         IStatementExecutorContext executorsCtx = application.getStatementExecutorContext();
-        JobId jobId = executorsCtx.getJobIdFromClientContextId(contextId);
+        IClientRequest req = executorsCtx.get(contextId);
         RequestStatus status;
 
-        if (jobId == null) {
+        if (req == null) {
             LOGGER.log(Level.WARN, "No job found for context id " + contextId);
             status = RequestStatus.NOT_FOUND;
         } else {
             try {
-                IHyracksClientConnection hcc = application.getHcc();
-                hcc.cancelJob(jobId);
-                executorsCtx.removeJobIdFromClientContextId(contextId);
+                req.cancel(appCtx);
+                executorsCtx.remove(contextId);
                 status = RequestStatus.SUCCESS;
             } catch (Exception e) {
                 LOGGER.log(Level.WARN, "unexpected exception thrown from cancel", e);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index 88b5da8..9835b68 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -30,6 +30,7 @@
 import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.api.IStatementExecutorContext;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -47,7 +48,6 @@
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
-import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.ResultProperties;
 import org.apache.asterix.translator.SessionConfig;
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 469a2dd..3b6a184 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
@@ -53,12 +53,14 @@
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.common.api.IMetadataLockManager;
+import org.apache.asterix.common.api.IStatementExecutorContext;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
 import org.apache.asterix.common.config.DatasetConfig.TransactionState;
 import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.context.ClientJobRequest;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
@@ -161,7 +163,6 @@
 import org.apache.asterix.translator.ExecutionPlansHtmlPrintUtil;
 import org.apache.asterix.translator.IRequestParameters;
 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;
@@ -2601,6 +2602,7 @@
     private static void createAndRunJob(IHyracksClientConnection hcc, EnumSet<JobFlag> jobFlags, Mutable<JobId> jId,
             IStatementCompiler compiler, IMetadataLocker locker, ResultDelivery resultDelivery, IResultPrinter printer,
             String clientContextId, IStatementExecutorContext ctx) throws Exception {
+        ClientJobRequest req = null;
         locker.lock();
         try {
             final JobSpecification jobSpec = compiler.compile();
@@ -2609,7 +2611,8 @@
             }
             final JobId jobId = JobUtils.runJob(hcc, jobSpec, jobFlags, false);
             if (ctx != null && clientContextId != null) {
-                ctx.put(clientContextId, jobId); // Adds the running job into the context.
+                req = new ClientJobRequest(ctx, clientContextId, jobId);
+                ctx.put(clientContextId, req); // Adds the running job into the context.
             }
             if (jId != null) {
                 jId.setValue(jobId);
@@ -2624,8 +2627,8 @@
         } finally {
             locker.unlock();
             // No matter the job succeeds or fails, removes it into the context.
-            if (ctx != null && clientContextId != null) {
-                ctx.removeJobIdFromClientContextId(clientContextId);
+            if (req != null) {
+                req.complete();
             }
         }
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index b180ef0..3f1fb6c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -36,12 +36,12 @@
 import org.apache.asterix.api.http.ctx.StatementExecutorContext;
 import org.apache.asterix.api.http.server.ActiveStatsApiServlet;
 import org.apache.asterix.api.http.server.ApiServlet;
+import org.apache.asterix.api.http.server.CcQueryCancellationServlet;
 import org.apache.asterix.api.http.server.ClusterApiServlet;
 import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
 import org.apache.asterix.api.http.server.DiagnosticsApiServlet;
 import org.apache.asterix.api.http.server.NodeControllerDetailsApiServlet;
-import org.apache.asterix.api.http.server.QueryCancellationServlet;
 import org.apache.asterix.api.http.server.QueryResultApiServlet;
 import org.apache.asterix.api.http.server.QueryServiceServlet;
 import org.apache.asterix.api.http.server.QueryStatusApiServlet;
@@ -55,6 +55,7 @@
 import org.apache.asterix.app.replication.NcLifecycleCoordinator;
 import org.apache.asterix.common.api.AsterixThreadFactory;
 import org.apache.asterix.common.api.INodeJobTracker;
+import org.apache.asterix.common.api.IStatementExecutorContext;
 import org.apache.asterix.common.cluster.IGlobalRecoveryManager;
 import org.apache.asterix.common.config.AsterixExtension;
 import org.apache.asterix.common.config.ExtensionProperties;
@@ -77,7 +78,6 @@
 import org.apache.asterix.metadata.lock.MetadataLockManager;
 import org.apache.asterix.runtime.job.resource.JobCapacityController;
 import org.apache.asterix.runtime.utils.CcApplicationContext;
-import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.util.MetadataBuiltinFunctions;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
@@ -271,7 +271,7 @@
     protected IServlet createServlet(ConcurrentMap<String, Object> ctx, String key, String... paths) {
         switch (key) {
             case Servlets.RUNNING_REQUESTS:
-                return new QueryCancellationServlet(ctx, paths);
+                return new CcQueryCancellationServlet(ctx, appCtx, paths);
             case Servlets.QUERY_STATUS:
                 return new QueryStatusApiServlet(ctx, appCtx, paths);
             case Servlets.QUERY_RESULT:
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
index d5262cf..2970105 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
@@ -28,9 +28,11 @@
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.asterix.api.http.ctx.StatementExecutorContext;
-import org.apache.asterix.api.http.server.QueryCancellationServlet;
+import org.apache.asterix.api.http.server.CcQueryCancellationServlet;
 import org.apache.asterix.api.http.server.ServletConstants;
-import org.apache.asterix.translator.IStatementExecutorContext;
+import org.apache.asterix.common.api.IStatementExecutorContext;
+import org.apache.asterix.common.context.ClientJobRequest;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.http.api.IServletRequest;
@@ -46,16 +48,17 @@
 
     @Test
     public void testDelete() throws Exception {
+        ICcApplicationContext appCtx = mock(ICcApplicationContext.class);
         // Creates a query cancellation servlet.
-        QueryCancellationServlet cancellationServlet =
-                new QueryCancellationServlet(new ConcurrentHashMap<>(), new String[] { "/" });
+        CcQueryCancellationServlet cancellationServlet =
+                new CcQueryCancellationServlet(new ConcurrentHashMap<>(), appCtx, new String[] { "/" });
         // Adds mocked Hyracks client connection into the servlet context.
         IHyracksClientConnection mockHcc = mock(IHyracksClientConnection.class);
         cancellationServlet.ctx().put(ServletConstants.HYRACKS_CONNECTION_ATTR, mockHcc);
         // Adds a query context into the servlet context.
         IStatementExecutorContext queryCtx = new StatementExecutorContext();
         cancellationServlet.ctx().put(ServletConstants.RUNNING_QUERIES_ATTR, queryCtx);
-
+        Mockito.when(appCtx.getHcc()).thenReturn(mockHcc);
         // Tests the case that query is not in the map.
         IServletRequest mockRequest = mockRequest("1");
         IServletResponse mockResponse = mock(IServletResponse.class);
@@ -63,7 +66,7 @@
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.NOT_FOUND);
 
         // Tests the case that query is in the map.
-        queryCtx.put("1", new JobId(1));
+        queryCtx.put("1", new ClientJobRequest(queryCtx, "1", new JobId(1)));
         cancellationServlet.handle(mockRequest, mockResponse);
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.OK);
 
@@ -73,7 +76,7 @@
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.BAD_REQUEST);
 
         // Tests the case that the job cancellation hit some exception from Hyracks.
-        queryCtx.put("2", new JobId(2));
+        queryCtx.put("2", new ClientJobRequest(queryCtx, "2", new JobId(2)));
         Mockito.doThrow(new Exception()).when(mockHcc).cancelJob(any());
         mockRequest = mockRequest("2");
         cancellationServlet.handle(mockRequest, mockResponse);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
new file mode 100644
index 0000000..30759de
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.common.api;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IClientRequest {
+
+    /**
+     * Mark the request as complete, non-cancellable anymore
+     */
+    void complete();
+
+    /**
+     * Cancel a request
+     *
+     * @param appCtx
+     * @throws HyracksDataException
+     */
+    void cancel(ICcApplicationContext appCtx) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IStatementExecutorContext.java
similarity index 69%
rename from asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
rename to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IStatementExecutorContext.java
index 81e1ebf..8cbc21b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IStatementExecutorContext.java
@@ -17,34 +17,31 @@
  * under the License.
  */
 
-package org.apache.asterix.translator;
-
-import org.apache.hyracks.api.job.JobId;
+package org.apache.asterix.common.api;
 
 /**
- * The context for statement executors, which maintains the meta information of all queries.
- * TODO(yingyi): also maintain the mapping from server generated request ids to jobs.
+ * The context for statement executors. Maintains ongoing user requests.
  */
 public interface IStatementExecutorContext {
 
     /**
-     * Gets the Hyracks JobId from the user-provided client context id.
+     * Gets the client request from the user-provided client context id.
      *
      * @param clientContextId,
      *            a user provided client context id.
-     * @return the Hyracks job id of class {@link org.apache.hyracks.api.job.JobId}.
+     * @return the client request
      */
-    JobId getJobIdFromClientContextId(String clientContextId);
+    IClientRequest get(String clientContextId);
 
     /**
-     * Puts a client context id for a statement and the corresponding Hyracks job id.
+     * Puts a client context id for a statement and the corresponding request.
      *
      * @param clientContextId,
      *            a user provided client context id.
-     * @param jobId,
+     * @param req,
      *            the Hyracks job id of class {@link org.apache.hyracks.api.job.JobId}.
      */
-    void put(String clientContextId, JobId jobId);
+    void put(String clientContextId, IClientRequest req);
 
     /**
      * Removes the information about the query corresponding to a user-provided client context id.
@@ -52,5 +49,5 @@
      * @param clientContextId,
      *            a user provided client context id.
      */
-    JobId removeJobIdFromClientContextId(String clientContextId);
+    IClientRequest remove(String clientContextId);
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseClientRequest.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseClientRequest.java
new file mode 100644
index 0000000..d5b625b
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseClientRequest.java
@@ -0,0 +1,55 @@
+/*
+ * 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.common.context;
+
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.api.IStatementExecutorContext;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class BaseClientRequest implements IClientRequest {
+    protected final IStatementExecutorContext ctx;
+    protected final String contextId;
+    private boolean complete;
+
+    public BaseClientRequest(IStatementExecutorContext ctx, String contextId) {
+        this.ctx = ctx;
+        this.contextId = contextId;
+    }
+
+    @Override
+    public synchronized void complete() {
+        if (complete) {
+            return;
+        }
+        complete = true;
+        ctx.remove(contextId);
+    }
+
+    @Override
+    public synchronized void cancel(ICcApplicationContext appCtx) throws HyracksDataException {
+        if (complete) {
+            return;
+        }
+        complete();
+        doCancel(appCtx);
+    }
+
+    protected abstract void doCancel(ICcApplicationContext appCtx) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ClientJobRequest.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ClientJobRequest.java
new file mode 100644
index 0000000..fb1150d
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ClientJobRequest.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.common.context;
+
+import org.apache.asterix.common.api.IStatementExecutorContext;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+
+public class ClientJobRequest extends BaseClientRequest {
+    private final JobId jobId;
+
+    public ClientJobRequest(IStatementExecutorContext ctx, String clientCtxId, JobId jobId) {
+        super(ctx, clientCtxId);
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doCancel(ICcApplicationContext appCtx) throws HyracksDataException {
+        IHyracksClientConnection hcc = appCtx.getHcc();
+        try {
+            hcc.cancelJob(jobId);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        ctx.remove(contextId);
+    }
+}

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

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

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 5: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/7wkkUu : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4541/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/269/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3976/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/292/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

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

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

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................

[NO ISSUE][MISC] Introduce IClientRequest

- user model changes: no
- storage format changes: no
- interface changes: yes
  + IClientRequest: used to represent a client request
    that can be cancelled.

Details:
- Introduce IClientRequest to allow for multiple types of requests
  to be cancellable.

Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
R asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IStatementExecutorContext.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseClientRequest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ClientJobRequest.java
16 files changed, 211 insertions(+), 69 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/74/2774/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2774
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/270/ (11/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/2774/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
File asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java:

PS3, Line 42: IStatementExecutorContext
> Why is the IStatementExecutorContext in a different package than the IState
The only reason is that it didn't feel algebraic enough to me... will move back


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1981/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/264/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/1980/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 6: Contrib+1

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3910/ (3/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3940/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7950/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3884/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4193/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Compilation Failed
https://goo.gl/VCZXhW : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Compilation Failed
https://goo.gl/d4x41Q : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/279/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2008/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3925/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

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

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

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................

[NO ISSUE][MISC] Introduce IClientRequest

- user model changes: no
- storage format changes: no
- interface changes: yes
  + IClientRequest: used to represent a client request
    that can be cancelled.

Details:
- Introduce IClientRequest to allow for multiple types of requests
  to be cancellable.

Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
R asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IStatementExecutorContext.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/BaseClientRequest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/ClientJobRequest.java
16 files changed, 208 insertions(+), 66 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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>

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3960/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4513/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/248/ (11/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2023/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4514/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/4011/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 2: -Contrib

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7915/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/3970/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/HDhGzX : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4569/ (13/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 4: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/GFoZyH : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3: Contrib-2

BAD Compatibility Tests Failed

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9483/ (3/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/251/ (11/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/258/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7909/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3882/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3934/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4219/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4512/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/263/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4234/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4556/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/304/ (13/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Till Westmann, Jenkins,

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

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

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................

[NO ISSUE][MISC] Introduce IClientRequest

- user model changes: no
- storage format changes: no
- interface changes: yes
  + IClientRequest: used to represent a client request
    that can be cancelled.

Details:
- Introduce IClientRequest to allow for multiple types of requests
  to be cancellable.

Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
15 files changed, 197 insertions(+), 58 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/74/2774/6
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2774
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3975/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3890/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4192/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/265/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3932/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5: Code-Review+2

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/2774/5/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java:

PS5, Line 50: clientContextId
not here anymore


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3883/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3933/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

(3 comments)

https://asterix-gerrit.ics.uci.edu/#/c/2774/3/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java:

PS3, Line 56: clientContextId
The clientContextId is part of the requestParameters already.


https://asterix-gerrit.ics.uci.edu/#/c/2774/3/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
File asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java:

PS3, Line 27: IStatementExecutorContext
Is this being used in this file?


https://asterix-gerrit.ics.uci.edu/#/c/2774/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
File asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java:

PS3, Line 42: IStatementExecutorContext
Why is the IStatementExecutorContext in a different package than the IStatementExecutor?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/3973/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 6:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 6
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4191/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


Patch Set 3: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/3MXDZs : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 3
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][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9441/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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][MISC] Introduce IClientRequest
......................................................................


[NO ISSUE][MISC] Introduce IClientRequest

- user model changes: no
- storage format changes: no
- interface changes: yes
  + IClientRequest: used to represent a client request
    that can be cancelled.

Details:
- Introduce IClientRequest to allow for multiple types of requests
  to be cancellable.

Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2774
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <ba...@gmail.com>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
15 files changed, 197 insertions(+), 58 deletions(-)

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



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
index 15267aa..a37f802 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
@@ -22,6 +22,7 @@
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -45,10 +46,11 @@
      * @param requestParameters
      * @param metadataProvider
      * @param resultSetId
+     * @param executorCtx
      * @throws HyracksDataException
      * @throws AlgebricksException
      */
     public abstract void handle(IHyracksClientConnection hcc, IStatementExecutor statementExecutor,
-            IRequestParameters requestParameters, MetadataProvider metadataProvider, int resultSetId)
-            throws HyracksDataException, AlgebricksException;
+            IRequestParameters requestParameters, MetadataProvider metadataProvider, int resultSetId,
+            IStatementExecutorContext executorCtx) throws HyracksDataException, AlgebricksException;
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
new file mode 100644
index 0000000..a9bd856
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
@@ -0,0 +1,54 @@
+/*
+ * 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.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public abstract class BaseClientRequest implements IClientRequest {
+    protected final IStatementExecutorContext ctx;
+    protected final String contextId;
+    private boolean complete;
+
+    public BaseClientRequest(IStatementExecutorContext ctx, String contextId) {
+        this.ctx = ctx;
+        this.contextId = contextId;
+    }
+
+    @Override
+    public synchronized void complete() {
+        if (complete) {
+            return;
+        }
+        complete = true;
+        ctx.remove(contextId);
+    }
+
+    @Override
+    public synchronized void cancel(ICcApplicationContext appCtx) throws HyracksDataException {
+        if (complete) {
+            return;
+        }
+        complete();
+        doCancel(appCtx);
+    }
+
+    protected abstract void doCancel(ICcApplicationContext appCtx) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java
new file mode 100644
index 0000000..520ce03
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java
@@ -0,0 +1,44 @@
+/*
+ * 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.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+
+public class ClientJobRequest extends BaseClientRequest {
+    private final JobId jobId;
+
+    public ClientJobRequest(IStatementExecutorContext ctx, String clientCtxId, JobId jobId) {
+        super(ctx, clientCtxId);
+        this.jobId = jobId;
+    }
+
+    @Override
+    protected void doCancel(ICcApplicationContext appCtx) throws HyracksDataException {
+        IHyracksClientConnection hcc = appCtx.getHcc();
+        try {
+            hcc.cancelJob(jobId);
+        } catch (Exception e) {
+            throw HyracksDataException.create(e);
+        }
+        ctx.remove(contextId);
+    }
+}
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
index 81e1ebf..78080f3 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
@@ -19,32 +19,31 @@
 
 package org.apache.asterix.translator;
 
-import org.apache.hyracks.api.job.JobId;
+import org.apache.asterix.common.api.IClientRequest;
 
 /**
- * The context for statement executors, which maintains the meta information of all queries.
- * TODO(yingyi): also maintain the mapping from server generated request ids to jobs.
+ * The context for statement executors. Maintains ongoing user requests.
  */
 public interface IStatementExecutorContext {
 
     /**
-     * Gets the Hyracks JobId from the user-provided client context id.
+     * Gets the client request from the user-provided client context id.
      *
      * @param clientContextId,
      *            a user provided client context id.
-     * @return the Hyracks job id of class {@link org.apache.hyracks.api.job.JobId}.
+     * @return the client request
      */
-    JobId getJobIdFromClientContextId(String clientContextId);
+    IClientRequest get(String clientContextId);
 
     /**
-     * Puts a client context id for a statement and the corresponding Hyracks job id.
+     * Puts a client context id for a statement and the corresponding request.
      *
      * @param clientContextId,
      *            a user provided client context id.
-     * @param jobId,
+     * @param req,
      *            the Hyracks job id of class {@link org.apache.hyracks.api.job.JobId}.
      */
-    void put(String clientContextId, JobId jobId);
+    void put(String clientContextId, IClientRequest req);
 
     /**
      * Removes the information about the query corresponding to a user-provided client context id.
@@ -52,5 +51,5 @@
      * @param clientContextId,
      *            a user provided client context id.
      */
-    JobId removeJobIdFromClientContextId(String clientContextId);
+    IClientRequest remove(String clientContextId);
 }
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
index ec181bb..c4e2859 100644
--- 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
@@ -18,7 +18,7 @@
  */
 package org.apache.asterix.translator;
 
-import org.apache.hyracks.api.job.JobId;
+import org.apache.asterix.common.api.IClientRequest;
 
 public class NoOpStatementExecutorContext implements IStatementExecutorContext {
 
@@ -28,17 +28,17 @@
     }
 
     @Override
-    public JobId getJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest get(String clientContextId) {
         return null;
     }
 
     @Override
-    public void put(String clientContextId, JobId jobId) {
+    public void put(String clientContextId, IClientRequest req) {
         // Dummy for when a statement doesn't support cancellation
     }
 
     @Override
-    public JobId removeJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest remove(String clientContextId) {
         return null;
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
index 7c06762..a4da189 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
@@ -16,31 +16,30 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-
 package org.apache.asterix.api.http.ctx;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
+import org.apache.asterix.common.api.IClientRequest;
 import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.job.JobId;
 
 public class StatementExecutorContext implements IStatementExecutorContext {
 
-    private final Map<String, JobId> runningQueries = new ConcurrentHashMap<>();
+    private final Map<String, IClientRequest> runningQueries = new ConcurrentHashMap<>();
 
     @Override
-    public JobId getJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest get(String clientContextId) {
         return runningQueries.get(clientContextId);
     }
 
     @Override
-    public void put(String clientContextId, JobId jobId) {
-        runningQueries.put(clientContextId, jobId);
+    public void put(String clientContextId, IClientRequest req) {
+        runningQueries.put(clientContextId, req);
     }
 
     @Override
-    public JobId removeJobIdFromClientContextId(String clientContextId) {
+    public IClientRequest remove(String clientContextId) {
         return runningQueries.remove(clientContextId);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
similarity index 76%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
index a8b3aef..5f5692d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
@@ -22,9 +22,9 @@
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.api.http.server.QueryServiceServlet.Parameter;
+import org.apache.asterix.common.api.IClientRequest;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
-import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.AbstractServlet;
@@ -37,11 +37,14 @@
 /**
  * The servlet provides a REST API for cancelling an on-going query.
  */
-public class QueryCancellationServlet extends AbstractServlet {
+public class CcQueryCancellationServlet extends AbstractServlet {
     private static final Logger LOGGER = LogManager.getLogger();
+    private final ICcApplicationContext appCtx;
 
-    public QueryCancellationServlet(ConcurrentMap<String, Object> ctx, String... paths) {
+    public CcQueryCancellationServlet(ConcurrentMap<String, Object> ctx, ICcApplicationContext appCtx,
+            String... paths) {
         super(ctx, paths);
+        this.appCtx = appCtx;
     }
 
     @Override
@@ -51,23 +54,17 @@
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
         }
-
-        // Retrieves the corresponding Hyracks job id.
-        IStatementExecutorContext runningQueries =
+        IStatementExecutorContext executorCtx =
                 (IStatementExecutorContext) ctx.get(ServletConstants.RUNNING_QUERIES_ATTR);
-        IHyracksClientConnection hcc = (IHyracksClientConnection) ctx.get(ServletConstants.HYRACKS_CONNECTION_ATTR);
-        JobId jobId = runningQueries.getJobIdFromClientContextId(clientContextId);
-
-        if (jobId == null) {
+        IClientRequest req = executorCtx.get(clientContextId);
+        if (req == null) {
             // response: NOT FOUND
             response.setStatus(HttpResponseStatus.NOT_FOUND);
             return;
         }
         try {
             // Cancels the on-going job.
-            hcc.cancelJob(jobId);
-            // Removes the cancelled query from the map activeQueries.
-            runningQueries.removeJobIdFromClientContextId(clientContextId);
+            req.cancel(appCtx);
             // response: OK
             response.setStatus(HttpResponseStatus.OK);
         } catch (Exception e) {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
index 5ad451f..60806d3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
@@ -32,6 +32,7 @@
 import org.apache.hyracks.api.application.INCServiceContext;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
+import org.apache.hyracks.http.server.AbstractServlet;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -41,7 +42,7 @@
 /**
  * The servlet provides a REST API on an NC for cancelling an on-going query.
  */
-public class NCQueryCancellationServlet extends QueryCancellationServlet {
+public class NCQueryCancellationServlet extends AbstractServlet {
     private static final Logger LOGGER = LogManager.getLogger();
     private final INCServiceContext serviceCtx;
     private final INCMessageBroker messageBroker;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index ce6185e..316646d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -53,9 +53,9 @@
 import org.apache.asterix.translator.ExecutionPlansJsonPrintUtil;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.asterix.translator.IStatementExecutor.ResultDelivery;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
-import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.ResultProperties;
 import org.apache.asterix.translator.SessionConfig;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
index a1c87d8..943aad3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
@@ -18,15 +18,14 @@
  */
 package org.apache.asterix.app.message;
 
+import org.apache.asterix.common.api.IClientRequest;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.common.messaging.api.ICcAddressedMessage;
 import org.apache.asterix.common.utils.RequestStatus;
 import org.apache.asterix.hyracks.bootstrap.CCApplication;
 import org.apache.asterix.messaging.CCMessageBroker;
 import org.apache.asterix.translator.IStatementExecutorContext;
-import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.control.cc.ClusterControllerService;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
@@ -51,17 +50,16 @@
         ClusterControllerService ccs = (ClusterControllerService) appCtx.getServiceContext().getControllerService();
         CCApplication application = (CCApplication) ccs.getApplication();
         IStatementExecutorContext executorsCtx = application.getStatementExecutorContext();
-        JobId jobId = executorsCtx.getJobIdFromClientContextId(contextId);
+        IClientRequest req = executorsCtx.get(contextId);
         RequestStatus status;
 
-        if (jobId == null) {
+        if (req == null) {
             LOGGER.log(Level.WARN, "No job found for context id " + contextId);
             status = RequestStatus.NOT_FOUND;
         } else {
             try {
-                IHyracksClientConnection hcc = application.getHcc();
-                hcc.cancelJob(jobId);
-                executorsCtx.removeJobIdFromClientContextId(contextId);
+                req.cancel(appCtx);
+                executorsCtx.remove(contextId);
                 status = RequestStatus.SUCCESS;
             } catch (Exception e) {
                 LOGGER.log(Level.WARN, "unexpected exception thrown from cancel", e);
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 469a2dd..0d884ba 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
@@ -152,6 +152,7 @@
 import org.apache.asterix.om.types.TypeSignature;
 import org.apache.asterix.transaction.management.service.transaction.DatasetIdFactory;
 import org.apache.asterix.translator.AbstractLangTranslator;
+import org.apache.asterix.translator.ClientJobRequest;
 import org.apache.asterix.translator.CompiledStatements.CompiledDeleteStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledInsertStatement;
 import org.apache.asterix.translator.CompiledStatements.CompiledLoadFromFileStatement;
@@ -408,7 +409,7 @@
                         break;
                     case EXTENSION:
                         ((ExtensionStatement) stmt).handle(hcc, this, requestParameters, metadataProvider,
-                                resultSetIdCounter);
+                                resultSetIdCounter, ctx);
                         break;
                     default:
                         throw new CompilationException(ErrorCode.COMPILATION_ILLEGAL_STATE, stmt.getSourceLocation(),
@@ -2601,6 +2602,7 @@
     private static void createAndRunJob(IHyracksClientConnection hcc, EnumSet<JobFlag> jobFlags, Mutable<JobId> jId,
             IStatementCompiler compiler, IMetadataLocker locker, ResultDelivery resultDelivery, IResultPrinter printer,
             String clientContextId, IStatementExecutorContext ctx) throws Exception {
+        ClientJobRequest req = null;
         locker.lock();
         try {
             final JobSpecification jobSpec = compiler.compile();
@@ -2609,7 +2611,8 @@
             }
             final JobId jobId = JobUtils.runJob(hcc, jobSpec, jobFlags, false);
             if (ctx != null && clientContextId != null) {
-                ctx.put(clientContextId, jobId); // Adds the running job into the context.
+                req = new ClientJobRequest(ctx, clientContextId, jobId);
+                ctx.put(clientContextId, req); // Adds the running job into the context.
             }
             if (jId != null) {
                 jId.setValue(jobId);
@@ -2624,8 +2627,8 @@
         } finally {
             locker.unlock();
             // No matter the job succeeds or fails, removes it into the context.
-            if (ctx != null && clientContextId != null) {
-                ctx.removeJobIdFromClientContextId(clientContextId);
+            if (req != null) {
+                req.complete();
             }
         }
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
index b180ef0..f4d24e2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
@@ -36,12 +36,12 @@
 import org.apache.asterix.api.http.ctx.StatementExecutorContext;
 import org.apache.asterix.api.http.server.ActiveStatsApiServlet;
 import org.apache.asterix.api.http.server.ApiServlet;
+import org.apache.asterix.api.http.server.CcQueryCancellationServlet;
 import org.apache.asterix.api.http.server.ClusterApiServlet;
 import org.apache.asterix.api.http.server.ClusterControllerDetailsApiServlet;
 import org.apache.asterix.api.http.server.ConnectorApiServlet;
 import org.apache.asterix.api.http.server.DiagnosticsApiServlet;
 import org.apache.asterix.api.http.server.NodeControllerDetailsApiServlet;
-import org.apache.asterix.api.http.server.QueryCancellationServlet;
 import org.apache.asterix.api.http.server.QueryResultApiServlet;
 import org.apache.asterix.api.http.server.QueryServiceServlet;
 import org.apache.asterix.api.http.server.QueryStatusApiServlet;
@@ -271,7 +271,7 @@
     protected IServlet createServlet(ConcurrentMap<String, Object> ctx, String key, String... paths) {
         switch (key) {
             case Servlets.RUNNING_REQUESTS:
-                return new QueryCancellationServlet(ctx, paths);
+                return new CcQueryCancellationServlet(ctx, appCtx, paths);
             case Servlets.QUERY_STATUS:
                 return new QueryStatusApiServlet(ctx, appCtx, paths);
             case Servlets.QUERY_RESULT:
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
index d5262cf..eae82af 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
@@ -28,8 +28,10 @@
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.asterix.api.http.ctx.StatementExecutorContext;
-import org.apache.asterix.api.http.server.QueryCancellationServlet;
+import org.apache.asterix.api.http.server.CcQueryCancellationServlet;
 import org.apache.asterix.api.http.server.ServletConstants;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.translator.ClientJobRequest;
 import org.apache.asterix.translator.IStatementExecutorContext;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.job.JobId;
@@ -46,16 +48,17 @@
 
     @Test
     public void testDelete() throws Exception {
+        ICcApplicationContext appCtx = mock(ICcApplicationContext.class);
         // Creates a query cancellation servlet.
-        QueryCancellationServlet cancellationServlet =
-                new QueryCancellationServlet(new ConcurrentHashMap<>(), new String[] { "/" });
+        CcQueryCancellationServlet cancellationServlet =
+                new CcQueryCancellationServlet(new ConcurrentHashMap<>(), appCtx, new String[] { "/" });
         // Adds mocked Hyracks client connection into the servlet context.
         IHyracksClientConnection mockHcc = mock(IHyracksClientConnection.class);
         cancellationServlet.ctx().put(ServletConstants.HYRACKS_CONNECTION_ATTR, mockHcc);
         // Adds a query context into the servlet context.
         IStatementExecutorContext queryCtx = new StatementExecutorContext();
         cancellationServlet.ctx().put(ServletConstants.RUNNING_QUERIES_ATTR, queryCtx);
-
+        Mockito.when(appCtx.getHcc()).thenReturn(mockHcc);
         // Tests the case that query is not in the map.
         IServletRequest mockRequest = mockRequest("1");
         IServletResponse mockResponse = mock(IServletResponse.class);
@@ -63,7 +66,7 @@
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.NOT_FOUND);
 
         // Tests the case that query is in the map.
-        queryCtx.put("1", new JobId(1));
+        queryCtx.put("1", new ClientJobRequest(queryCtx, "1", new JobId(1)));
         cancellationServlet.handle(mockRequest, mockResponse);
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.OK);
 
@@ -73,7 +76,7 @@
         verify(mockResponse, times(1)).setStatus(HttpResponseStatus.BAD_REQUEST);
 
         // Tests the case that the job cancellation hit some exception from Hyracks.
-        queryCtx.put("2", new JobId(2));
+        queryCtx.put("2", new ClientJobRequest(queryCtx, "2", new JobId(2)));
         Mockito.doThrow(new Exception()).when(mockHcc).cancelJob(any());
         mockRequest = mockRequest("2");
         cancellationServlet.handle(mockRequest, mockResponse);
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
new file mode 100644
index 0000000..30759de
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.common.api;
+
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IClientRequest {
+
+    /**
+     * Mark the request as complete, non-cancellable anymore
+     */
+    void complete();
+
+    /**
+     * Cancel a request
+     *
+     * @param appCtx
+     * @throws HyracksDataException
+     */
+    void cancel(ICcApplicationContext appCtx) throws HyracksDataException;
+}
diff --git a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
index 7f8d31d..f2be6cd 100644
--- a/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
+++ b/asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
@@ -1545,6 +1545,7 @@
         }
     }
 
+    @Override
     public AsterixTupleFilterFactory createTupleFilterFactory(IOperatorSchema[] inputSchemas,
             IVariableTypeEnvironment typeEnv, ILogicalExpression filterExpr, JobGenContext context)
             throws AlgebricksException {

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 7
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4520/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 4
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

Posted by "abdullah alamoudi (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Jenkins,

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

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

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................

[NO ISSUE][MISC] Introduce IClientRequest

- user model changes: no
- storage format changes: no
- interface changes: yes
  + IClientRequest: used to represent a client request
    that can be cancelled.

Details:
- Introduce IClientRequest to allow for multiple types of requests
  to be cancellable.

Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/algebra/extension/ExtensionStatement.java
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/BaseClientRequest.java
A asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/ClientJobRequest.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorContext.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/NoOpStatementExecutorContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/ctx/StatementExecutorContext.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/CcQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryCancellationServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/CancelQueryRequest.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/CCApplication.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/QueryCancellationServletTest.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IClientRequest.java
M asterixdb/asterix-metadata/src/main/java/org/apache/asterix/metadata/declared/MetadataProvider.java
15 files changed, 198 insertions(+), 58 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/74/2774/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2774
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 5: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
Gerrit-PatchSet: 5
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: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][MISC] Introduce IClientRequest

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

Change subject: [NO ISSUE][MISC] Introduce IClientRequest
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7908/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f65da1744ea7ecf26ea3f8a576ebaf4472ccd62
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-HasComments: No