You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org> on 2017/10/13 15:38:01 UTC

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

Murtadha Hubail has uploaded a new change for review.

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................

[ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn

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

Details:
- Do not reset the primary index operation tracker active operations
  count if the metadata transaction was aborted.
- Add test cases.

Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
A asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
4 files changed, 129 insertions(+), 1 deletion(-)


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

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 71c67f4..f5e94b1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -121,6 +121,10 @@
         this.ncs = nodeControllers.toArray(new NodeControllerService[nodeControllers.size()]);
     }
 
+    public ClusterControllerService getClusterControllerService() {
+        return cc;
+    }
+
     protected CCConfig createCCConfig(ConfigManager configManager) throws IOException {
         CCConfig ccConfig = new CCConfig(configManager);
         ccConfig.setClusterListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
new file mode 100644
index 0000000..dd4b110
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.test.metadata;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.NodeGroup;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class MetadataTxnTest {
+
+    private static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration.xml";
+    private static final TestExecutor testExecutor = new TestExecutor();
+    private static final AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
+
+    @Before
+    public void setUp() throws Exception {
+        System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
+        integrationUtil.init(true);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        integrationUtil.deinit(true);
+    }
+
+    @Test
+    public void abortMetadataTxn() throws Exception {
+        ICcApplicationContext appCtx =
+                (ICcApplicationContext) integrationUtil.getClusterControllerService().getApplicationContext();
+        final MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
+        final MetadataTransactionContext mdTxn = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxn);
+        try {
+            final List<String> ngNodes = Arrays.asList("asterix_nc1");
+            MetadataManager.INSTANCE.addNodegroup(mdTxn, new NodeGroup("ng", ngNodes));
+            MetadataManager.INSTANCE.abortTransaction(mdTxn);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+    }
+
+    @Test
+    public void rebalanceFailureMetadataTxn() throws Exception {
+        ICcApplicationContext appCtx =
+                (ICcApplicationContext) integrationUtil.getClusterControllerService().getApplicationContext();
+        String nodeGroup = "ng";
+        String datasetName = "dataset1";
+        final TestCaseContext.OutputFormat format = TestCaseContext.OutputFormat.CLEAN_JSON;
+        // create original node group
+        testExecutor.executeSqlppUpdateOrDdl("CREATE nodegroup " + nodeGroup + " on asterix_nc2;", format);
+        // create original dataset
+        testExecutor.executeSqlppUpdateOrDdl("CREATE TYPE KeyType AS { id: int };", format);
+        testExecutor.executeSqlppUpdateOrDdl(
+                "CREATE DATASET " + datasetName + "(KeyType) PRIMARY KEY id on " + nodeGroup + ";", format);
+        // find source dataset
+        Dataset sourceDataset;
+        MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
+        final MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            sourceDataset = metadataProvider.findDataset(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME, datasetName);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+
+        // create rebalance metadata provider and metadata txn
+        metadataProvider = new MetadataProvider(appCtx, null);
+        final MetadataTransactionContext rebalanceTxn = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(rebalanceTxn);
+        try {
+            final Set<String> rebalanceToNodes = Stream.of("asterix_nc1").collect(Collectors.toSet());
+            DatasetUtil.createNodeGroupForNewDataset(sourceDataset.getDataverseName(), sourceDataset.getDatasetName(),
+                    sourceDataset.getRebalanceCount() + 1, rebalanceToNodes, metadataProvider);
+            // rebalance failed --> abort txn
+            MetadataManager.INSTANCE.abortTransaction(rebalanceTxn);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+        // ensure original dataset can be dropped after rebalance failure
+        testExecutor.executeSqlppUpdateOrDdl("DROP DATASET " + datasetName + ";", format);
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index 67b25b6..0899c21 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -184,6 +184,9 @@
     public void cleanupNumActiveOperationsForAbortedJob(int numberOfActiveOperations) {
         numberOfActiveOperations *= -1;
         numActiveOperations.getAndAdd(numberOfActiveOperations);
+        if (numActiveOperations.get() < 0) {
+            throw new IllegalStateException("The number of active operations cannot be negative!");
+        }
     }
 
     public boolean isFlushOnExit() {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
index f53aeb1..eb37f22 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -147,7 +147,11 @@
     @Override
     public void notifyOptracker(boolean isJobLevelCommit) {
         try {
-            if (isJobLevelCommit && isMetadataTxn) {
+            /**
+             * in case of transaction abort {@link TransactionContext#cleanupForAbort()} will
+             * clean the primaryIndexOpTracker state.
+             */
+            if (isJobLevelCommit && isMetadataTxn && txnState.get() != ITransactionManager.ABORTED) {
                 primaryIndexOpTracker.exclusiveJobCommitted();
             } else if (!isJobLevelCommit) {
                 primaryIndexOpTracker.completeOperation(null, LSMOperationType.MODIFICATION, null,

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2: -Integration-Tests

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/2071/1/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
File asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java:

PS1, Line 61: abortMetadataTxn
> This test is not checking whether the node group was added or not.
Done


PS1, Line 114: / ensure original dataset can be dropped after rebalance failure
> also check the node group associated with the dataset?
We drop the node group by default if all datasets created on it are dropped. I added the check that the node group was dropped too.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/2108/ (6/10)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/5267/ (10/10)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/6763/ (1/10)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2: Contrib+1

Analytics Compatibility Tests Successful Build 2758
: SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2: Integration-Tests-1

Integration Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/4272/ : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


[ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn

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

Details:
- Do not reset the primary index operation tracker active operations
  count if the metadata transaction was aborted.
- Add test cases.

Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2071
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Ian Maxon <im...@apache.org>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: abdullah alamoudi <ba...@gmail.com>
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
A asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
4 files changed, 152 insertions(+), 1 deletion(-)

Approvals:
  Anon. E. Moose #1000171: 
  abdullah alamoudi: Looks good to me, approved
  Jenkins: Verified; No violations found; ; Verified
  Ian Maxon: Looks good to me, but someone else must approve



diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
index 71c67f4..f5e94b1 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
@@ -121,6 +121,10 @@
         this.ncs = nodeControllers.toArray(new NodeControllerService[nodeControllers.size()]);
     }
 
+    public ClusterControllerService getClusterControllerService() {
+        return cc;
+    }
+
     protected CCConfig createCCConfig(ConfigManager configManager) throws IOException {
         CCConfig ccConfig = new CCConfig(configManager);
         ccConfig.setClusterListenAddress(Inet4Address.getLoopbackAddress().getHostAddress());
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
new file mode 100644
index 0000000..3969ec5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.test.metadata;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
+import org.apache.asterix.common.config.GlobalConfig;
+import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.metadata.MetadataManager;
+import org.apache.asterix.metadata.MetadataTransactionContext;
+import org.apache.asterix.metadata.bootstrap.MetadataBuiltinEntities;
+import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.metadata.entities.Dataset;
+import org.apache.asterix.metadata.entities.NodeGroup;
+import org.apache.asterix.metadata.utils.DatasetUtil;
+import org.apache.asterix.test.common.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class MetadataTxnTest {
+
+    private static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration.xml";
+    private static final TestExecutor testExecutor = new TestExecutor();
+    private static final AsterixHyracksIntegrationUtil integrationUtil = new AsterixHyracksIntegrationUtil();
+
+    @Before
+    public void setUp() throws Exception {
+        System.setProperty(GlobalConfig.CONFIG_FILE_PROPERTY, TEST_CONFIG_FILE_NAME);
+        integrationUtil.init(true);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        integrationUtil.deinit(true);
+    }
+
+    @Test
+    public void abortMetadataTxn() throws Exception {
+        ICcApplicationContext appCtx =
+                (ICcApplicationContext) integrationUtil.getClusterControllerService().getApplicationContext();
+        final MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
+        final MetadataTransactionContext mdTxn = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxn);
+        final String nodeGroupName = "ng";
+        try {
+            final List<String> ngNodes = Arrays.asList("asterix_nc1");
+            MetadataManager.INSTANCE.addNodegroup(mdTxn, new NodeGroup(nodeGroupName, ngNodes));
+            MetadataManager.INSTANCE.abortTransaction(mdTxn);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+
+        // ensure that the node group was not added
+        final MetadataTransactionContext readMdTxn = MetadataManager.INSTANCE.beginTransaction();
+        try {
+            final NodeGroup nodegroup = MetadataManager.INSTANCE.getNodegroup(readMdTxn, nodeGroupName);
+            if (nodegroup != null) {
+                throw new AssertionError("nodegroup was found after metadata txn was aborted");
+            }
+        } finally {
+            MetadataManager.INSTANCE.commitTransaction(readMdTxn);
+        }
+    }
+
+    @Test
+    public void rebalanceFailureMetadataTxn() throws Exception {
+        ICcApplicationContext appCtx =
+                (ICcApplicationContext) integrationUtil.getClusterControllerService().getApplicationContext();
+        String nodeGroup = "ng";
+        String datasetName = "dataset1";
+        final TestCaseContext.OutputFormat format = TestCaseContext.OutputFormat.CLEAN_JSON;
+        // create original node group
+        testExecutor.executeSqlppUpdateOrDdl("CREATE nodegroup " + nodeGroup + " on asterix_nc2;", format);
+        // create original dataset
+        testExecutor.executeSqlppUpdateOrDdl("CREATE TYPE KeyType AS { id: int };", format);
+        testExecutor.executeSqlppUpdateOrDdl(
+                "CREATE DATASET " + datasetName + "(KeyType) PRIMARY KEY id on " + nodeGroup + ";", format);
+        // find source dataset
+        Dataset sourceDataset;
+        MetadataProvider metadataProvider = new MetadataProvider(appCtx, null);
+        final MetadataTransactionContext mdTxnCtx = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(mdTxnCtx);
+        try {
+            sourceDataset = metadataProvider.findDataset(MetadataBuiltinEntities.DEFAULT_DATAVERSE_NAME, datasetName);
+            MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+
+        // create rebalance metadata provider and metadata txn
+        metadataProvider = new MetadataProvider(appCtx, null);
+        final MetadataTransactionContext rebalanceTxn = MetadataManager.INSTANCE.beginTransaction();
+        metadataProvider.setMetadataTxnContext(rebalanceTxn);
+        try {
+            final Set<String> rebalanceToNodes = Stream.of("asterix_nc1").collect(Collectors.toSet());
+            DatasetUtil.createNodeGroupForNewDataset(sourceDataset.getDataverseName(), sourceDataset.getDatasetName(),
+                    sourceDataset.getRebalanceCount() + 1, rebalanceToNodes, metadataProvider);
+            // rebalance failed --> abort txn
+            MetadataManager.INSTANCE.abortTransaction(rebalanceTxn);
+        } finally {
+            metadataProvider.getLocks().unlock();
+        }
+        // ensure original dataset can be dropped after rebalance failure
+        testExecutor.executeSqlppUpdateOrDdl("DROP DATASET " + datasetName + ";", format);
+
+        // ensure the node group was dropped too since its only dataset was dropped
+        final MetadataTransactionContext readMdTxn = MetadataManager.INSTANCE.beginTransaction();
+        try {
+            final NodeGroup nodegroup = MetadataManager.INSTANCE.getNodegroup(readMdTxn, nodeGroup);
+            if (nodegroup != null) {
+                throw new AssertionError("nodegroup was found after its only dataset was dropped");
+            }
+        } finally {
+            MetadataManager.INSTANCE.commitTransaction(readMdTxn);
+        }
+    }
+}
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
index 67b25b6..0899c21 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
@@ -184,6 +184,9 @@
     public void cleanupNumActiveOperationsForAbortedJob(int numberOfActiveOperations) {
         numberOfActiveOperations *= -1;
         numActiveOperations.getAndAdd(numberOfActiveOperations);
+        if (numActiveOperations.get() < 0) {
+            throw new IllegalStateException("The number of active operations cannot be negative!");
+        }
     }
 
     public boolean isFlushOnExit() {
diff --git a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
index f53aeb1..eb37f22 100644
--- a/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -147,7 +147,11 @@
     @Override
     public void notifyOptracker(boolean isJobLevelCommit) {
         try {
-            if (isJobLevelCommit && isMetadataTxn) {
+            /**
+             * in case of transaction abort {@link TransactionContext#cleanupForAbort()} will
+             * clean the primaryIndexOpTracker state.
+             */
+            if (isJobLevelCommit && isMetadataTxn && txnState.get() != ITransactionManager.ABORTED) {
                 primaryIndexOpTracker.exclusiveJobCommitted();
             } else if (!isJobLevelCommit) {
                 primaryIndexOpTracker.completeOperation(null, LSMOperationType.MODIFICATION, null,

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

Gerrit-MessageType: merged
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/6767/ (1/10)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

Posted by "Murtadha Hubail (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/2071

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................

[ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn

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

Details:
- Do not reset the primary index operation tracker active operations
  count if the metadata transaction was aborted.
- Add test cases.

Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/AsterixHyracksIntegrationUtil.java
A asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/context/PrimaryIndexOperationTracker.java
M asterixdb/asterix-transactions/src/main/java/org/apache/asterix/transaction/management/service/transaction/TransactionContext.java
4 files changed, 152 insertions(+), 1 deletion(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/1219/ (10/10)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2: Code-Review+1

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Ian Maxon <im...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
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]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app/2113/ (2/10)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1: Contrib+1

Analytics Compatibility Tests Successful Build 2754
: SUCCESS

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

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

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 1:

(2 comments)

This is a good start.

https://asterix-gerrit.ics.uci.edu/#/c/2071/1/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java
File asterixdb/asterix-app/src/test/java/org/apache/asterix/test/metadata/MetadataTxnTest.java:

PS1, Line 61: abortMetadataTxn
This test is not checking whether the node group was added or not.


PS1, Line 114: / ensure original dataset can be dropped after rebalance failure
also check the node group associated with the dataset?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Met...

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

Change subject: [ASTERIXDB-2131][TX] Do Not Reset Active Ops For Aborted Metadata Txn
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: Iee47aca1be0675b704ed9f176d9e10daef1cfc7f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: abdullah alamoudi <ba...@gmail.com>
Gerrit-HasComments: No