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/02/24 07:34:29 UTC

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

Murtadha Hubail has uploaded a new change for review.

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................

Exceptions Cleanup for Replication/FaultTolerance Strategies

Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
15 files changed, 76 insertions(+), 31 deletions(-)


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

diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index 2efb139..8549a34 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -182,7 +182,9 @@
     @Override
     public void startLocalRecovery(Set<Integer> partitions) throws IOException, ACIDException {
         state = SystemState.RECOVERING;
-        LOGGER.log(Level.INFO, "starting recovery ...");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("starting recovery ...");
+        }
 
         long readableSmallestLSN = logMgr.getReadableSmallestLSN();
         Checkpoint checkpointObject = checkpointManager.getLatest();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
index e6638e8..21f1670 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
@@ -46,7 +46,9 @@
 import org.apache.asterix.common.api.INCLifecycleTask;
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.replication.IFaultToleranceStrategy;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
@@ -129,7 +131,7 @@
                 process((ReplayPartitionLogsResponseMessage) message);
                 break;
             default:
-                throw new HyracksDataException("Unsupported message type: " + message.getType().name());
+                throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
         }
     }
 
@@ -141,7 +143,9 @@
 
     private synchronized void process(ReplayPartitionLogsResponseMessage msg) {
         hotStandbyMetadataReplica.add(msg.getNodeId());
-        LOGGER.log(Level.INFO, "Hot Standby Metadata Replicas: " + hotStandbyMetadataReplica);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Hot Standby Metadata Replicas: " + hotStandbyMetadataReplica);
+        }
     }
 
     private synchronized void process(StartupTaskRequestMessage msg) throws HyracksDataException {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
index 0ee4f6a..51defaa 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
@@ -40,7 +40,9 @@
 import org.apache.asterix.common.api.INCLifecycleTask;
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.replication.IFaultToleranceStrategy;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
@@ -81,7 +83,7 @@
                 process((NCLifecycleTaskReportMessage) message);
                 break;
             default:
-                throw new HyracksDataException("Unsupported message type: " + message.getType().name());
+                throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
         }
     }
 
@@ -118,7 +120,9 @@
             }
             clusterManager.refreshState();
         } else {
-            LOGGER.log(Level.SEVERE, msg.getNodeId() + " failed to complete startup. ", msg.getException());
+            if (LOGGER.isLoggable(Level.SEVERE)) {
+                LOGGER.log(Level.SEVERE, msg.getNodeId() + " failed to complete startup. ", msg.getException());
+            }
         }
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
index 0a9a215..0ab4e54 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
@@ -59,7 +59,9 @@
                 try {
                     messageBroker.sendApplicationMessageToNC(msg, replica);
                 } catch (Exception e) {
-                    LOGGER.log(Level.WARNING, "Failed sending an application message to an NC", e);
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.log(Level.WARNING, "Failed sending an application message to an NC", e);
+                    }
                 }
             }
         }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index f5d97ba..ae40827 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -59,6 +59,7 @@
 import org.apache.asterix.testframework.context.TestFileContext;
 import org.apache.asterix.testframework.xml.TestCase.CompilationUnit;
 import org.apache.asterix.testframework.xml.TestGroup;
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.commons.lang3.StringUtils;
@@ -1119,8 +1120,7 @@
         String config = actual.toString();
         ObjectMapper om = new ObjectMapper();
         String logDir = om.readTree(config).findPath("transaction.log.dirs").get(nodeId).asText();
-        ProcessBuilder pb = new ProcessBuilder("rm", "-rf", logDir);
-        pb.start().waitFor();
+        FileUtils.deleteQuietly(new File(logDir));
     }
 
     public void executeTest(String actualPath, TestCaseContext testCaseCtx, ProcessBuilder pb,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
index 980ad24..1ba9471 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.event.schema.cluster.Node;
 import org.apache.asterix.event.schema.cluster.Replica;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ClusterProperties {
 
@@ -83,7 +84,7 @@
         return -1;
     }
 
-    public IReplicationStrategy getReplicationStrategy() {
+    public IReplicationStrategy getReplicationStrategy() throws HyracksDataException {
         return ReplicationStrategyFactory.create(cluster);
     }
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
index cf2ce4f..116609e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.common.replication.Replica;
 import org.apache.asterix.event.schema.cluster.Cluster;
 import org.apache.asterix.event.schema.cluster.Node;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.StorageUtil;
 import org.apache.hyracks.util.StorageUtil.StorageUnit;
 
@@ -54,7 +55,7 @@
     private final Cluster cluster;
     private final IReplicationStrategy repStrategy;
 
-    public ReplicationProperties(PropertiesAccessor accessor) {
+    public ReplicationProperties(PropertiesAccessor accessor) throws HyracksDataException {
         super(accessor);
         this.cluster = ClusterProperties.INSTANCE.getCluster();
         this.repStrategy = ClusterProperties.INSTANCE.getReplicationStrategy();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 83ff3a2..1221955 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -18,7 +18,6 @@
  */
 package org.apache.asterix.common.exceptions;
 
-import java.io.File;
 import java.io.InputStream;
 import java.util.Map;
 
@@ -37,6 +36,9 @@
     // Extension errors
     public static final int EXTENSION_ID_CONFLICT = 4001;
     public static final int EXTENSION_COMPONENT_CONFLICT = 4002;
+    public static final int UNSUPPORTED_MESSAGE_TYPE = 4003;
+    public static final int INVALID_CONFIGURATION = 4004;
+    public static final int UNSUPPORTED_REPLICATION_STRATEGY = 4005;
 
     // Runtime errors
     public static final int CASTING_FIELD = 1;
@@ -52,6 +54,7 @@
     public static final int OUT_OF_BOUND = 11;
     public static final int COERCION = 12;
     public static final int DUPLICATE_FIELD_NAME = 13;
+    public static final int INITIALIZATION_ERROR = 14;
 
     // Compilation errors
     public static final int PARSE_ERROR = 1001;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
index ad326b2..dc69383 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
@@ -25,7 +25,10 @@
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.config.ClusterProperties;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.event.schema.cluster.Cluster;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ChainedDeclusteringReplicationStrategy implements IReplicationStrategy {
 
@@ -45,7 +48,9 @@
         int nodeIndex = ClusterProperties.INSTANCE.getNodeIndex(nodeId);
 
         if (nodeIndex == -1) {
-            LOGGER.log(Level.WARNING, "Could not find node " + nodeId + " in cluster configurations");
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("Could not find node " + nodeId + " in cluster configurations");
+            }
             return Collections.emptySet();
         }
 
@@ -74,9 +79,9 @@
     }
 
     @Override
-    public ChainedDeclusteringReplicationStrategy from(Cluster cluster) {
+    public ChainedDeclusteringReplicationStrategy from(Cluster cluster) throws HyracksDataException {
         if (cluster.getHighAvailability().getDataReplication().getReplicationFactor() == null) {
-            throw new IllegalStateException("Replication factor must be specified.");
+            throw new RuntimeDataException(ErrorCode.INVALID_CONFIGURATION, "Replication factor must be specified.");
         }
         ChainedDeclusteringReplicationStrategy cd = new ChainedDeclusteringReplicationStrategy();
         cd.replicationFactor = cluster.getHighAvailability().getDataReplication().getReplicationFactor().intValue();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
index f65f6ac..b3f1701 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
@@ -21,6 +21,7 @@
 import java.util.Set;
 
 import org.apache.asterix.event.schema.cluster.Cluster;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IReplicationStrategy {
 
@@ -54,5 +55,5 @@
      * @param cluster
      * @return A replication strategy based on the passed configurations.
      */
-    IReplicationStrategy from(Cluster cluster);
+    IReplicationStrategy from(Cluster cluster) throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
index 711f06d..6682489 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
@@ -24,9 +24,12 @@
 import java.util.Set;
 
 import org.apache.asterix.common.config.ClusterProperties;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.event.schema.cluster.Cluster;
 import org.apache.asterix.event.schema.cluster.Node;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class MetadataOnlyReplicationStrategy implements IReplicationStrategy {
 
@@ -57,9 +60,9 @@
     }
 
     @Override
-    public MetadataOnlyReplicationStrategy from(Cluster cluster) {
+    public MetadataOnlyReplicationStrategy from(Cluster cluster) throws HyracksDataException {
         if (cluster.getMetadataNode() == null) {
-            throw new IllegalStateException("Metadata node must be specified.");
+            throw new RuntimeDataException(ErrorCode.INVALID_CONFIGURATION, "Metadata node must be specified.");
         }
 
         Node metadataNode = ClusterProperties.INSTANCE.getNodeById(cluster.getMetadataNode());
@@ -70,14 +73,14 @@
         if (cluster.getHighAvailability().getFaultTolerance().getReplica() == null
                 || cluster.getHighAvailability().getFaultTolerance().getReplica().getNodeId() == null
                 || cluster.getHighAvailability().getFaultTolerance().getReplica().getNodeId().isEmpty()) {
-            throw new IllegalStateException("One or more replicas must be specified for metadata node.");
+            throw new RuntimeDataException(ErrorCode.INVALID_CONFIGURATION, "One or more replicas must be specified for metadata node.");
         }
 
         final Set<Replica> replicas = new HashSet<>();
         for (String nodeId : cluster.getHighAvailability().getFaultTolerance().getReplica().getNodeId()) {
             Node node = ClusterProperties.INSTANCE.getNodeById(nodeId);
             if (node == null) {
-                throw new IllegalStateException("Invalid replica specified: " + nodeId);
+                throw new RuntimeDataException(ErrorCode.INVALID_CONFIGURATION, "Invalid replica specified: " + nodeId);
             }
             replicas.add(new Replica(node));
         }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
index b61b38a..4794e37 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
@@ -21,7 +21,10 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.event.schema.cluster.Cluster;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ReplicationStrategyFactory {
 
@@ -38,7 +41,7 @@
         throw new AssertionError();
     }
 
-    public static IReplicationStrategy create(Cluster cluster) {
+    public static IReplicationStrategy create(Cluster cluster) throws HyracksDataException {
         boolean highAvailabilityEnabled = cluster.getHighAvailability() != null
                 && cluster.getHighAvailability().getEnabled() != null
                 && Boolean.valueOf(cluster.getHighAvailability().getEnabled());
@@ -49,14 +52,15 @@
         }
         String strategyName = cluster.getHighAvailability().getDataReplication().getStrategy().toLowerCase();
         if (!BUILT_IN_REPLICATION_STRATEGY.containsKey(strategyName)) {
-            throw new IllegalArgumentException(String.format("Unsupported Replication Strategy. Available types: %s",
-                    BUILT_IN_REPLICATION_STRATEGY.keySet().toString()));
+            throw new RuntimeDataException(ErrorCode.UNSUPPORTED_REPLICATION_STRATEGY,
+                    String.format(strategyName + ". Available strategies: %s",
+                            BUILT_IN_REPLICATION_STRATEGY.keySet().toString()));
         }
         Class<? extends IReplicationStrategy> clazz = BUILT_IN_REPLICATION_STRATEGY.get(strategyName);
         try {
             return clazz.newInstance().from(cluster);
         } catch (InstantiationException | IllegalAccessException e) {
-            throw new IllegalStateException(e);
+            throw new RuntimeDataException(ErrorCode.INITIALIZATION_ERROR, e);
         }
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
index 3e85276..52144d2e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
@@ -34,11 +34,11 @@
 public interface IRecoveryManager {
 
     public enum SystemState {
-        INITIAL_RUN,
-        NEW_UNIVERSE,
-        RECOVERING,
-        HEALTHY,
-        CORRUPTED
+        INITIAL_RUN, // The first time the NC is bootstrapped.
+        NEW_UNIVERSE, // No checkpoint files found on NC and it is not the INITIAL_RUN (data loss).
+        RECOVERING, // Recovery process is on-going.
+        HEALTHY, // All txn logs effects are on disk (no need to perform recovery).
+        CORRUPTED // Some txn logs need to be replayed (need to perform recover).
     }
 
     public class ResourceType {
@@ -99,7 +99,6 @@
      *
      * @param partitions
      * @param lowWaterMarkLSN
-     * @param failedNode
      * @throws IOException
      * @throws ACIDException
      */
@@ -122,5 +121,12 @@
      */
     public void deleteRecoveryTemporaryFiles();
 
+    /**
+     * Performs the local recovery process on {@code partitions}
+     *
+     * @param partitions
+     * @throws IOException
+     * @throws ACIDException
+     */
     void startLocalRecovery(Set<Integer> partitions) throws IOException, ACIDException;
 }
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 3e96972..95e9e1c 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -27,6 +27,9 @@
 # For the extension lifecycle
 4001 = Two Extensions share the same Id: %1$s
 4002 = Extension Conflict between %1$s and %2$s both extensions extend %3$s
+4003 = Unsupported message type: %1$s
+4004 = Invalid configuration: %1$s
+4005 = Unsupported replication strategy %1$s
 
 # Type errors
 2,1002 = Type mismatch: function %1$s expects its %2$s input parameter to be type %3$s, but the actual input type is %4$s
@@ -44,6 +47,9 @@
 10 = Invalid value: function %1$s expects its %2$s input parameter to be a non-negative value, but gets %3$s
 11 = Index out of bound in %1$s: %2$s
 12 = Invalid implicit scalar to collection coercion in %1$s
+
+# Initialization errors
+14 = Initialization error
 
 # Compile-time check errors
 1007 = Invalid expression: function %1$s expects its %2$s input parameter to be a %3$s expression, but the actual expression is %4$s
@@ -142,4 +148,4 @@
 3075 = Closed field %1$s has null value.
 3076 = %1$s: no files found
 3077 = %1$s: path not found
-3078 = Cannot obtain hdfs scheduler
+3078 = Cannot obtain hdfs scheduler
\ No newline at end of file
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
index 2db6114..7a0a797 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.installer.test;
 
 import java.io.File;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Map;
@@ -41,7 +42,9 @@
 @RunWith(Parameterized.class)
 public class MetadataReplicationIT {
 
-    private static final String PATH_BASE = "src/test/resources/integrationts/metadata_only_replication/";
+    private static final String PATH_BASE =
+            Paths.get("src", "test", "resources", "integrationts", "metadata_only_replication").toString()
+                    + File.separator;
     private static final String PATH_ACTUAL = "target" + File.separator + "ittest" + File.separator;
     private static final Logger LOGGER = Logger.getLogger(MetadataReplicationIT.class.getName());
     private static String reportPath = new File(

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

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

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

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................

Exceptions Cleanup for Replication/FaultTolerance Strategies

Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/MetadataBootstrapTask.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
19 files changed, 90 insertions(+), 44 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 4: BAD-1

BAD Compatibility Tests Failed

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4429/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Exceptions Cleanup for Replication/FaultTolerance Strategies

Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1529
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Yingyi Bu <bu...@gmail.com>
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/MetadataBootstrapTask.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
19 files changed, 90 insertions(+), 44 deletions(-)

Approvals:
  Yingyi Bu: Looks good to me, approved
  Jenkins: Verified; No violations found; Verified

Objections:
  Jenkins: Violations found



diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
index b4b7a95..625f18f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
@@ -191,7 +191,7 @@
 
         IRecoveryManager recoveryMgr = txnSubsystem.getRecoveryManager();
         SystemState systemState = recoveryMgr.getSystemState();
-        if (initialRun || systemState == SystemState.NEW_UNIVERSE) {
+        if (initialRun || systemState == SystemState.PERMANENT_DATA_LOSS) {
             //delete any storage data before the resource factory is initialized
             localResourceRepository.deleteStorageData(true);
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
index 2efb139..4ee1122 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
@@ -121,10 +121,10 @@
         Checkpoint checkpointObject = checkpointManager.getLatest();
         if (checkpointObject == null) {
             //The checkpoint file doesn't exist => Failure happened during NC initialization.
-            //Retry to initialize the NC by setting the state to NEW_UNIVERSE
-            state = SystemState.NEW_UNIVERSE;
+            //Retry to initialize the NC by setting the state to PERMANENT_DATA_LOSS
+            state = SystemState.PERMANENT_DATA_LOSS;
             if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("The checkpoint file doesn't exist: systemState = NEW_UNIVERSE");
+                LOGGER.info("The checkpoint file doesn't exist: systemState = PERMANENT_DATA_LOSS");
             }
             return state;
         }
@@ -182,7 +182,9 @@
     @Override
     public void startLocalRecovery(Set<Integer> partitions) throws IOException, ACIDException {
         state = SystemState.RECOVERING;
-        LOGGER.log(Level.INFO, "starting recovery ...");
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("starting recovery ...");
+        }
 
         long readableSmallestLSN = logMgr.getReadableSmallestLSN();
         Checkpoint checkpointObject = checkpointManager.getLatest();
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/MetadataBootstrapTask.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/MetadataBootstrapTask.java
index b6d85d9..ab19573 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/MetadataBootstrapTask.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/MetadataBootstrapTask.java
@@ -36,7 +36,7 @@
         IAppRuntimeContext appContext = (IAppRuntimeContext) ncs.getApplicationContext().getApplicationObject();
         try {
             SystemState state = appContext.getTransactionSubsystem().getRecoveryManager().getSystemState();
-            appContext.initializeMetadata(state == SystemState.NEW_UNIVERSE);
+            appContext.initializeMetadata(state == SystemState.PERMANENT_DATA_LOSS);
         } catch (Exception e) {
             throw ExceptionUtils.convertToHyracksDataException(e);
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
index 084563f..5104610 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
@@ -55,7 +55,9 @@
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.ReplicationProperties;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.replication.IFaultToleranceStrategy;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
@@ -448,7 +450,7 @@
                 process((CompleteFailbackResponseMessage) message);
                 break;
             default:
-                throw new HyracksDataException("Unsupported message type: " + message.getType().name());
+                throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
         }
     }
 
@@ -485,7 +487,7 @@
         final String nodeId = msg.getNodeId();
         final SystemState state = msg.getState();
         List<INCLifecycleTask> tasks;
-        if (state == SystemState.INITIAL_RUN || state == SystemState.HEALTHY) {
+        if (state == SystemState.BOOTSTRAPPING || state == SystemState.HEALTHY) {
             tasks = buildStartupSequence(nodeId);
         } else {
             // failed node returned. Need to start failback process
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
index e6638e8..5a7036a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
@@ -46,7 +46,9 @@
 import org.apache.asterix.common.api.INCLifecycleTask;
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.replication.IFaultToleranceStrategy;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
@@ -129,7 +131,7 @@
                 process((ReplayPartitionLogsResponseMessage) message);
                 break;
             default:
-                throw new HyracksDataException("Unsupported message type: " + message.getType().name());
+                throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
         }
     }
 
@@ -141,7 +143,9 @@
 
     private synchronized void process(ReplayPartitionLogsResponseMessage msg) {
         hotStandbyMetadataReplica.add(msg.getNodeId());
-        LOGGER.log(Level.INFO, "Hot Standby Metadata Replicas: " + hotStandbyMetadataReplica);
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("Hot Standby Metadata Replicas: " + hotStandbyMetadataReplica);
+        }
     }
 
     private synchronized void process(StartupTaskRequestMessage msg) throws HyracksDataException {
@@ -202,7 +206,7 @@
     private List<INCLifecycleTask> buildParticipantStartupSequence(String nodeId, SystemState state) {
         final List<INCLifecycleTask> tasks = new ArrayList<>();
         switch (state) {
-            case NEW_UNIVERSE:
+            case PERMANENT_DATA_LOSS:
                 // If the metadata node (or replica) failed and lost its data
                 // => Metadata Remote Recovery from standby replica
                 tasks.add(getMetadataPartitionRecoveryPlan());
@@ -215,7 +219,7 @@
                         .stream().map(ClusterPartition::getPartitionId).collect(Collectors.toSet()));
                 tasks.add(rt);
                 break;
-            case INITIAL_RUN:
+            case BOOTSTRAPPING:
             case HEALTHY:
             case RECOVERING:
                 break;
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
index 0ee4f6a..51defaa 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
@@ -40,7 +40,9 @@
 import org.apache.asterix.common.api.INCLifecycleTask;
 import org.apache.asterix.common.cluster.ClusterPartition;
 import org.apache.asterix.common.cluster.IClusterStateManager;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.ExceptionUtils;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.messaging.api.ICCMessageBroker;
 import org.apache.asterix.common.replication.IFaultToleranceStrategy;
 import org.apache.asterix.common.replication.INCLifecycleMessage;
@@ -81,7 +83,7 @@
                 process((NCLifecycleTaskReportMessage) message);
                 break;
             default:
-                throw new HyracksDataException("Unsupported message type: " + message.getType().name());
+                throw new RuntimeDataException(ErrorCode.UNSUPPORTED_MESSAGE_TYPE, message.getType().name());
         }
     }
 
@@ -118,7 +120,9 @@
             }
             clusterManager.refreshState();
         } else {
-            LOGGER.log(Level.SEVERE, msg.getNodeId() + " failed to complete startup. ", msg.getException());
+            if (LOGGER.isLoggable(Level.SEVERE)) {
+                LOGGER.log(Level.SEVERE, msg.getNodeId() + " failed to complete startup. ", msg.getException());
+            }
         }
     }
 
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index ce57648..7f649bc 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -117,9 +117,9 @@
         IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
         systemState = recoveryMgr.getSystemState();
 
-        if (systemState == SystemState.NEW_UNIVERSE) {
+        if (systemState == SystemState.PERMANENT_DATA_LOSS) {
             if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("System state: " + SystemState.NEW_UNIVERSE);
+                LOGGER.info("System state: " + SystemState.PERMANENT_DATA_LOSS);
                 LOGGER.info("Node ID: " + nodeId);
                 LOGGER.info("Stores: " + PrintUtil.toString(metadataProperties.getStores()));
                 LOGGER.info("Root Metadata Store: " + metadataProperties.getStores().get(nodeId)[0]);
@@ -160,8 +160,8 @@
     @Override
     public void notifyStartupComplete() throws Exception {
         // Since we don't pass initial run flag in AsterixHyracksIntegrationUtil, we use the virtualNC flag
-        if (systemState == SystemState.NEW_UNIVERSE && (initialRun || virtualNC)) {
-            systemState = SystemState.INITIAL_RUN;
+        if (systemState == SystemState.PERMANENT_DATA_LOSS && (initialRun || virtualNC)) {
+            systemState = SystemState.BOOTSTRAPPING;
         }
         // Request startup tasks from CC
         StartupTaskRequestMessage.send((NodeControllerService) ncApplicationContext.getControllerService(),
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
index 0a9a215..0ab4e54 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
@@ -59,7 +59,9 @@
                 try {
                     messageBroker.sendApplicationMessageToNC(msg, replica);
                 } catch (Exception e) {
-                    LOGGER.log(Level.WARNING, "Failed sending an application message to an NC", e);
+                    if (LOGGER.isLoggable(Level.WARNING)) {
+                        LOGGER.log(Level.WARNING, "Failed sending an application message to an NC", e);
+                    }
                 }
             }
         }
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index f5d97ba..ae40827 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -59,6 +59,7 @@
 import org.apache.asterix.testframework.context.TestFileContext;
 import org.apache.asterix.testframework.xml.TestCase.CompilationUnit;
 import org.apache.asterix.testframework.xml.TestGroup;
+import org.apache.commons.io.FileUtils;
 import org.apache.commons.io.IOUtils;
 import org.apache.commons.io.output.ByteArrayOutputStream;
 import org.apache.commons.lang3.StringUtils;
@@ -1119,8 +1120,7 @@
         String config = actual.toString();
         ObjectMapper om = new ObjectMapper();
         String logDir = om.readTree(config).findPath("transaction.log.dirs").get(nodeId).asText();
-        ProcessBuilder pb = new ProcessBuilder("rm", "-rf", logDir);
-        pb.start().waitFor();
+        FileUtils.deleteQuietly(new File(logDir));
     }
 
     public void executeTest(String actualPath, TestCaseContext testCaseCtx, ProcessBuilder pb,
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
index 980ad24..1ba9471 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
@@ -32,6 +32,7 @@
 import org.apache.asterix.event.schema.cluster.Node;
 import org.apache.asterix.event.schema.cluster.Replica;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ClusterProperties {
 
@@ -83,7 +84,7 @@
         return -1;
     }
 
-    public IReplicationStrategy getReplicationStrategy() {
+    public IReplicationStrategy getReplicationStrategy() throws HyracksDataException {
         return ReplicationStrategyFactory.create(cluster);
     }
 
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
index cf2ce4f..116609e 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
@@ -25,6 +25,7 @@
 import org.apache.asterix.common.replication.Replica;
 import org.apache.asterix.event.schema.cluster.Cluster;
 import org.apache.asterix.event.schema.cluster.Node;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.util.StorageUtil;
 import org.apache.hyracks.util.StorageUtil.StorageUnit;
 
@@ -54,7 +55,7 @@
     private final Cluster cluster;
     private final IReplicationStrategy repStrategy;
 
-    public ReplicationProperties(PropertiesAccessor accessor) {
+    public ReplicationProperties(PropertiesAccessor accessor) throws HyracksDataException {
         super(accessor);
         this.cluster = ClusterProperties.INSTANCE.getCluster();
         this.repStrategy = ClusterProperties.INSTANCE.getReplicationStrategy();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 6dbafd1..70e0ae9 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -18,10 +18,10 @@
  */
 package org.apache.asterix.common.exceptions;
 
-import java.io.File;
 import java.io.InputStream;
 import java.util.Map;
 
+import org.apache.asterix.event.schema.cluster.FaultTolerance;
 import org.apache.hyracks.api.util.ErrorMessageUtil;
 
 // Error code:
@@ -37,6 +37,9 @@
     // Extension errors
     public static final int EXTENSION_ID_CONFLICT = 4001;
     public static final int EXTENSION_COMPONENT_CONFLICT = 4002;
+    public static final int UNSUPPORTED_MESSAGE_TYPE = 4003;
+    public static final int INVALID_CONFIGURATION = 4004;
+    public static final int UNSUPPORTED_REPLICATION_STRATEGY = 4005;
 
     // Runtime errors
     public static final int CASTING_FIELD = 1;
@@ -53,7 +56,7 @@
     public static final int COERCION = 12;
     public static final int DUPLICATE_FIELD_NAME = 13;
     public static final int PROPERTY_NOT_SET = 14;
-
+    public static final int INSTANTIATION_ERROR = 100;
 
     // Compilation errors
     public static final int PARSE_ERROR = 1001;
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
index ad326b2..dc69383 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
@@ -25,7 +25,10 @@
 import java.util.logging.Logger;
 
 import org.apache.asterix.common.config.ClusterProperties;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.event.schema.cluster.Cluster;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ChainedDeclusteringReplicationStrategy implements IReplicationStrategy {
 
@@ -45,7 +48,9 @@
         int nodeIndex = ClusterProperties.INSTANCE.getNodeIndex(nodeId);
 
         if (nodeIndex == -1) {
-            LOGGER.log(Level.WARNING, "Could not find node " + nodeId + " in cluster configurations");
+            if (LOGGER.isLoggable(Level.WARNING)) {
+                LOGGER.warning("Could not find node " + nodeId + " in cluster configurations");
+            }
             return Collections.emptySet();
         }
 
@@ -74,9 +79,9 @@
     }
 
     @Override
-    public ChainedDeclusteringReplicationStrategy from(Cluster cluster) {
+    public ChainedDeclusteringReplicationStrategy from(Cluster cluster) throws HyracksDataException {
         if (cluster.getHighAvailability().getDataReplication().getReplicationFactor() == null) {
-            throw new IllegalStateException("Replication factor must be specified.");
+            throw new RuntimeDataException(ErrorCode.INVALID_CONFIGURATION, "Replication factor must be specified.");
         }
         ChainedDeclusteringReplicationStrategy cd = new ChainedDeclusteringReplicationStrategy();
         cd.replicationFactor = cluster.getHighAvailability().getDataReplication().getReplicationFactor().intValue();
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
index f65f6ac..b3f1701 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
@@ -21,6 +21,7 @@
 import java.util.Set;
 
 import org.apache.asterix.event.schema.cluster.Cluster;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface IReplicationStrategy {
 
@@ -54,5 +55,5 @@
      * @param cluster
      * @return A replication strategy based on the passed configurations.
      */
-    IReplicationStrategy from(Cluster cluster);
+    IReplicationStrategy from(Cluster cluster) throws HyracksDataException;
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
index 711f06d..bd4b32f 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
@@ -24,9 +24,12 @@
 import java.util.Set;
 
 import org.apache.asterix.common.config.ClusterProperties;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.common.metadata.MetadataIndexImmutableProperties;
 import org.apache.asterix.event.schema.cluster.Cluster;
 import org.apache.asterix.event.schema.cluster.Node;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class MetadataOnlyReplicationStrategy implements IReplicationStrategy {
 
@@ -57,9 +60,9 @@
     }
 
     @Override
-    public MetadataOnlyReplicationStrategy from(Cluster cluster) {
+    public MetadataOnlyReplicationStrategy from(Cluster cluster) throws HyracksDataException {
         if (cluster.getMetadataNode() == null) {
-            throw new IllegalStateException("Metadata node must be specified.");
+            throw new RuntimeDataException(ErrorCode.INVALID_CONFIGURATION, "Metadata node must be specified.");
         }
 
         Node metadataNode = ClusterProperties.INSTANCE.getNodeById(cluster.getMetadataNode());
@@ -70,14 +73,15 @@
         if (cluster.getHighAvailability().getFaultTolerance().getReplica() == null
                 || cluster.getHighAvailability().getFaultTolerance().getReplica().getNodeId() == null
                 || cluster.getHighAvailability().getFaultTolerance().getReplica().getNodeId().isEmpty()) {
-            throw new IllegalStateException("One or more replicas must be specified for metadata node.");
+            throw new RuntimeDataException(ErrorCode.INVALID_CONFIGURATION,
+                    "One or more replicas must be specified for metadata node.");
         }
 
         final Set<Replica> replicas = new HashSet<>();
         for (String nodeId : cluster.getHighAvailability().getFaultTolerance().getReplica().getNodeId()) {
             Node node = ClusterProperties.INSTANCE.getNodeById(nodeId);
             if (node == null) {
-                throw new IllegalStateException("Invalid replica specified: " + nodeId);
+                throw new RuntimeDataException(ErrorCode.INVALID_CONFIGURATION, "Invalid replica specified: " + nodeId);
             }
             replicas.add(new Replica(node));
         }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
index b61b38a..703ddcc 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
@@ -21,7 +21,10 @@
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
 import org.apache.asterix.event.schema.cluster.Cluster;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class ReplicationStrategyFactory {
 
@@ -38,7 +41,7 @@
         throw new AssertionError();
     }
 
-    public static IReplicationStrategy create(Cluster cluster) {
+    public static IReplicationStrategy create(Cluster cluster) throws HyracksDataException {
         boolean highAvailabilityEnabled = cluster.getHighAvailability() != null
                 && cluster.getHighAvailability().getEnabled() != null
                 && Boolean.valueOf(cluster.getHighAvailability().getEnabled());
@@ -49,14 +52,15 @@
         }
         String strategyName = cluster.getHighAvailability().getDataReplication().getStrategy().toLowerCase();
         if (!BUILT_IN_REPLICATION_STRATEGY.containsKey(strategyName)) {
-            throw new IllegalArgumentException(String.format("Unsupported Replication Strategy. Available types: %s",
-                    BUILT_IN_REPLICATION_STRATEGY.keySet().toString()));
+            throw new RuntimeDataException(ErrorCode.UNSUPPORTED_REPLICATION_STRATEGY,
+                    String.format("%s. Available strategies: %s", strategyName,
+                            BUILT_IN_REPLICATION_STRATEGY.keySet().toString()));
         }
         Class<? extends IReplicationStrategy> clazz = BUILT_IN_REPLICATION_STRATEGY.get(strategyName);
         try {
             return clazz.newInstance().from(cluster);
         } catch (InstantiationException | IllegalAccessException e) {
-            throw new IllegalStateException(e);
+            throw new RuntimeDataException(ErrorCode.INSTANTIATION_ERROR, e, clazz.getName());
         }
     }
 }
\ No newline at end of file
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
index 3e85276..84e1019 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
@@ -34,11 +34,11 @@
 public interface IRecoveryManager {
 
     public enum SystemState {
-        INITIAL_RUN,
-        NEW_UNIVERSE,
-        RECOVERING,
-        HEALTHY,
-        CORRUPTED
+        BOOTSTRAPPING, // The first time the NC is bootstrapped.
+        PERMANENT_DATA_LOSS, // No checkpoint files found on NC and it is not BOOTSTRAPPING (data loss).
+        RECOVERING, // Recovery process is on-going.
+        HEALTHY, // All txn logs effects are on disk (no need to perform recovery).
+        CORRUPTED // Some txn logs need to be replayed (need to perform recover).
     }
 
     public class ResourceType {
@@ -99,7 +99,6 @@
      *
      * @param partitions
      * @param lowWaterMarkLSN
-     * @param failedNode
      * @throws IOException
      * @throws ACIDException
      */
@@ -122,5 +121,12 @@
      */
     public void deleteRecoveryTemporaryFiles();
 
+    /**
+     * Performs the local recovery process on {@code partitions}
+     *
+     * @param partitions
+     * @throws IOException
+     * @throws ACIDException
+     */
     void startLocalRecovery(Set<Integer> partitions) throws IOException, ACIDException;
 }
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index 14bd0c7..2bdb2a3 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -27,6 +27,9 @@
 # For the extension lifecycle
 4001 = Two Extensions share the same Id: %1$s
 4002 = Extension Conflict between %1$s and %2$s both extensions extend %3$s
+4003 = Unsupported message type: %1$s
+4004 = Invalid configuration: %1$s
+4005 = Unsupported replication strategy %1$s
 
 # Type errors
 2,1002 = Type mismatch: function %1$s expects its %2$s input parameter to be type %3$s, but the actual input type is %4$s
@@ -45,6 +48,7 @@
 11 = Index out of bound in %1$s: %2$s
 12 = Invalid implicit scalar to collection coercion in %1$s
 14 = Property %1$s not set
+100 = Unable to instantiate class %1$s
 
 # Compile-time check errors
 1007 = Invalid expression: function %1$s expects its %2$s input parameter to be a %3$s expression, but the actual expression is %4$s
diff --git a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
index 2db6114..7a0a797 100644
--- a/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
+++ b/asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
@@ -19,6 +19,7 @@
 package org.apache.asterix.installer.test;
 
 import java.io.File;
+import java.nio.file.Paths;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Map;
@@ -41,7 +42,9 @@
 @RunWith(Parameterized.class)
 public class MetadataReplicationIT {
 
-    private static final String PATH_BASE = "src/test/resources/integrationts/metadata_only_replication/";
+    private static final String PATH_BASE =
+            Paths.get("src", "test", "resources", "integrationts", "metadata_only_replication").toString()
+                    + File.separator;
     private static final String PATH_ACTUAL = "target" + File.separator + "ittest" + File.separator;
     private static final Logger LOGGER = Logger.getLogger(MetadataReplicationIT.class.getName());
     private static String reportPath = new File(

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 3:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1529/3/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java:

PS3, Line 38: PERMANENT_DATA_LOSS
This sounds rather harsh compared to "NEW_UNIVERSE" - a good change :)


https://asterix-gerrit.ics.uci.edu/#/c/1529/3/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
File asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties:

PS3, Line 51: Instantiation
Having a new "section" here seems a little too tight - especially as there probably many more "Data errors" and only a few "Instantiation errors" that still need to be added. Maybe we just start at 100?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 1: BAD-1

BAD Compatibility Tests Failed

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 1: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4427/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 3: BAD-1

BAD Compatibility Tests Failed

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

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

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

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................

Exceptions Cleanup for Replication/FaultTolerance Strategies

Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/NCAppRuntimeContext.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/task/MetadataBootstrapTask.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/AutoFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
19 files changed, 92 insertions(+), 44 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/29/1529/3
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/1529
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4465/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 4: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 2:

(2 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1529/2/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java:

PS2, Line 38: checkpoint
I'm confused here.
NEW_UNIVERSE doesn't mean that it's a fresh instance? It meant for that from the very beginning.

At least, the name "NEW_UNIVERSE" doesn't seem to match that state.

Can you use "NEW_UNIVERSE" for the current "INITIAL_RUN", and use "DATA_LOSS" for the current "NEW_UNIVERSE"?


https://asterix-gerrit.ics.uci.edu/#/c/1529/2/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
File asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties:

PS2, Line 52: error
A more concrete error message? A user still cannot know the root cause via this error message.


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

Posted by "Murtadha Hubail (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

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

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

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................

Exceptions Cleanup for Replication/FaultTolerance Strategies

Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
---
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/nc/RecoveryManager.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/MetadataNodeFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/replication/NoFaultToleranceStrategy.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/util/FaultToleranceUtil.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ClusterProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/ReplicationProperties.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ChainedDeclusteringReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/IReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/MetadataOnlyReplicationStrategy.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/replication/ReplicationStrategyFactory.java
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-installer/src/test/java/org/apache/asterix/installer/test/MetadataReplicationIT.java
15 files changed, 77 insertions(+), 31 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 2: BAD-1

BAD Compatibility Tests Failed

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/4473/

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 2:

(4 comments)

https://asterix-gerrit.ics.uci.edu/#/c/1529/2/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java:

PS2, Line 38: checkpoint
> I'm confused here.
I replaced INTIAL_RUN by BOOTSTRAPPING and NEW_UNIVERSE by PERMANENT_DATA_LOSS.


https://asterix-gerrit.ics.uci.edu/#/c/1529/3/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java
File asterixdb/asterix-common/src/main/java/org/apache/asterix/common/transactions/IRecoveryManager.java:

PS3, Line 38: NEW_UNIVERSE, // No
> This sounds rather harsh compared to "NEW_UNIVERSE" - a good change :)
It's good to see this dark side of you :)


https://asterix-gerrit.ics.uci.edu/#/c/1529/2/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
File asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties:

PS2, Line 52: error
> A more concrete error message? A user still cannot know the root cause via 
Done


https://asterix-gerrit.ics.uci.edu/#/c/1529/3/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
File asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties:

PS3, Line 51: Initializatio
> Having a new "section" here seems a little too tight - especially as there 
Done


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: Exceptions Cleanup for Replication/FaultTolerance Strategies

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

Change subject: Exceptions Cleanup for Replication/FaultTolerance Strategies
......................................................................


Patch Set 4: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I8f28b8db42bf7c8537ff2da22cbd2a97e243f32f
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <hu...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Yingyi Bu <bu...@gmail.com>
Gerrit-HasComments: No