1) support providing iodevices in cluster configuration
2) support for providing java_opts in cluster configuration
3) managix uses cluster/client ip in cluster configuration
4) consistent use of IPs and Ids when describing an asterix instance
5) managix poulates log.properties for the initialization of log manager and bakes them inside asterix-app*.jar 
   Log Manager can read log properties by its class loader.    
6) changes in  backup/restore to account for artifacts created by log manager. 
7) cosmetic changes.


git-svn-id: https://asterixdb.googlecode.com/svn/branches/asterix_lsm_stabilization@1449 eaa15691-b419-025a-1212-ee371bd00084
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
index 4e5fde1..8d38b56 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/hyracks/bootstrap/NCApplicationEntryPoint.java
@@ -18,132 +18,152 @@
 import edu.uci.ics.hyracks.api.application.INCApplicationEntryPoint;
 
 public class NCApplicationEntryPoint implements INCApplicationEntryPoint {
-    private static final Logger LOGGER = Logger.getLogger(NCApplicationEntryPoint.class.getName());
+	private static final Logger LOGGER = Logger
+			.getLogger(NCApplicationEntryPoint.class.getName());
 
-    private INCApplicationContext ncApplicationContext = null;
-    private AsterixAppRuntimeContext runtimeContext;
-    private String nodeId;
-    private boolean isMetadataNode = false;
-    private boolean stopInitiated = false;
-    private SystemState systemState = SystemState.NEW_UNIVERSE;
+	private INCApplicationContext ncApplicationContext = null;
+	private AsterixAppRuntimeContext runtimeContext;
+	private String nodeId;
+	private boolean isMetadataNode = false;
+	private boolean stopInitiated = false;
+	private SystemState systemState = SystemState.NEW_UNIVERSE;
 
-    @Override
-    public void start(INCApplicationContext ncAppCtx, String[] args) throws Exception {
-        ncApplicationContext = ncAppCtx;
-        nodeId = ncApplicationContext.getNodeId();
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Starting Asterix node controller: " + nodeId);
-        }
+	@Override
+	public void start(INCApplicationContext ncAppCtx, String[] args)
+			throws Exception {
+		ncApplicationContext = ncAppCtx;
+		nodeId = ncApplicationContext.getNodeId();
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("Starting Asterix node controller: " + nodeId);
+		}
 
-        runtimeContext = new AsterixAppRuntimeContext(ncApplicationContext);
-        runtimeContext.initialize();
-        ncApplicationContext.setApplicationObject(runtimeContext);
-        JVMShutdownHook sHook = new JVMShutdownHook(this);
-        Runtime.getRuntime().addShutdownHook(sHook);
+		runtimeContext = new AsterixAppRuntimeContext(ncApplicationContext);
+		runtimeContext.initialize();
+		ncApplicationContext.setApplicationObject(runtimeContext);
+		JVMShutdownHook sHook = new JVMShutdownHook(this);
+		Runtime.getRuntime().addShutdownHook(sHook);
 
-        //#. recover if the system is corrupted by checking system state.
-        IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
-        systemState = recoveryMgr.getSystemState();
-        if (systemState != SystemState.NEW_UNIVERSE) {
-            PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
-                    .getLocalResourceRepository();
-            localResourceRepository.initialize(nodeId, null, false, runtimeContext.getResourceIdFactory());
-        }
-        if (systemState == SystemState.CORRUPTED) {
-            recoveryMgr.startRecovery(true);
-        } else if (systemState == SystemState.NEW_UNIVERSE) {
-            recoveryMgr.checkpoint(true);
-        }
-    }
+		// #. recover if the system is corrupted by checking system state.
+		IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem()
+				.getRecoveryManager();
+		systemState = recoveryMgr.getSystemState();
+		if (systemState != SystemState.NEW_UNIVERSE) {
+			PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
+					.getLocalResourceRepository();
+			localResourceRepository.initialize(nodeId, null, false,
+					runtimeContext.getResourceIdFactory());
+		}
+		if (systemState == SystemState.CORRUPTED) {
+			recoveryMgr.startRecovery(true);
+		} else if (systemState == SystemState.NEW_UNIVERSE) {
+			recoveryMgr.checkpoint(true);
+		}
+	}
 
-    @Override
-    public void stop() throws Exception {
-        if (!stopInitiated) {
-            stopInitiated = true;
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Stopping Asterix node controller: " + nodeId);
-            }
+	@Override
+	public void stop() throws Exception {
+		if (!stopInitiated) {
+			stopInitiated = true;
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("Stopping Asterix node controller: " + nodeId);
+			}
 
-            IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
-            recoveryMgr.checkpoint(true);
+			IRecoveryManager recoveryMgr = runtimeContext
+					.getTransactionSubsystem().getRecoveryManager();
+			recoveryMgr.checkpoint(true);
 
-            if (isMetadataNode) {
-                MetadataBootstrap.stopUniverse();
-            }
-            runtimeContext.deinitialize();
-        } else {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Duplicate attempt to stop ignored: " + nodeId);
-            }
-        }
-    }
+			if (isMetadataNode) {
+				MetadataBootstrap.stopUniverse();
+			}
+			runtimeContext.deinitialize();
+		} else {
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("Duplicate attempt to stop ignored: " + nodeId);
+			}
+		}
+	}
 
-    @Override
-    public void notifyStartupComplete() throws Exception {
-        IAsterixStateProxy proxy = (IAsterixStateProxy) ncApplicationContext.getDistributedState();
+	@Override
+	public void notifyStartupComplete() throws Exception {
+		IAsterixStateProxy proxy = (IAsterixStateProxy) ncApplicationContext
+				.getDistributedState();
 
-        if (systemState == SystemState.NEW_UNIVERSE) {
-            PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
-                    .getLocalResourceRepository();
-            localResourceRepository.initialize(nodeId, proxy.getAsterixProperties().getStores().get(nodeId)[0], true,
-                    null);
-        }
+		if (systemState == SystemState.NEW_UNIVERSE) {
+			PersistentLocalResourceRepository localResourceRepository = (PersistentLocalResourceRepository) runtimeContext
+					.getLocalResourceRepository();
+			System.out.println("nodeid" + nodeId);
+			System.out.println("proxy" + proxy);
+			System.out.println("stores"
+					+ proxy.getAsterixProperties().getStores());
+			System.out.println("store"
+					+ proxy.getAsterixProperties().getStores().get(nodeId)[0]);
 
-        isMetadataNode = nodeId.equals(proxy.getAsterixProperties().getMetadataNodeName());
-        if (isMetadataNode) {
-            registerRemoteMetadataNode(proxy);
+			localResourceRepository.initialize(nodeId, proxy
+					.getAsterixProperties().getStores().get(nodeId)[0], true,
+					null);
+		}
 
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Bootstrapping metadata");
-            }
-            MetadataManager.INSTANCE = new MetadataManager(proxy);
-            MetadataManager.INSTANCE.init();
-            MetadataBootstrap.startUniverse(proxy.getAsterixProperties(), ncApplicationContext,
-                    systemState == SystemState.NEW_UNIVERSE);
-            MetadataBootstrap.startDDLRecovery();
-        }
+		isMetadataNode = nodeId.equals(proxy.getAsterixProperties()
+				.getMetadataNodeName());
+		if (isMetadataNode) {
+			registerRemoteMetadataNode(proxy);
 
-        IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem().getRecoveryManager();
-        recoveryMgr.checkpoint(true);
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("Bootstrapping metadata");
+			}
+			MetadataManager.INSTANCE = new MetadataManager(proxy);
+			MetadataManager.INSTANCE.init();
+			MetadataBootstrap.startUniverse(proxy.getAsterixProperties(),
+					ncApplicationContext,
+					systemState == SystemState.NEW_UNIVERSE);
+			MetadataBootstrap.startDDLRecovery();
+		}
 
-        //TODO
-        //reclaim storage for orphaned index artifacts in NCs.
-    }
+		IRecoveryManager recoveryMgr = runtimeContext.getTransactionSubsystem()
+				.getRecoveryManager();
+		recoveryMgr.checkpoint(true);
 
-    public void registerRemoteMetadataNode(IAsterixStateProxy proxy) throws RemoteException {
-        IMetadataNode stub = null;
-        MetadataNode.INSTANCE.initialize(runtimeContext);
-        stub = (IMetadataNode) UnicastRemoteObject.exportObject(MetadataNode.INSTANCE, 0);
-        proxy.setMetadataNode(stub);
+		// TODO
+		// reclaim storage for orphaned index artifacts in NCs.
+	}
 
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info("Metadata node bound");
-        }
-    }
+	public void registerRemoteMetadataNode(IAsterixStateProxy proxy)
+			throws RemoteException {
+		IMetadataNode stub = null;
+		MetadataNode.INSTANCE.initialize(runtimeContext);
+		stub = (IMetadataNode) UnicastRemoteObject.exportObject(
+				MetadataNode.INSTANCE, 0);
+		proxy.setMetadataNode(stub);
 
-    /**
-     * Shutdown hook that invokes {@link NCApplicationEntryPoint#stop() stop} method.
-     */
-    private static class JVMShutdownHook extends Thread {
+		if (LOGGER.isLoggable(Level.INFO)) {
+			LOGGER.info("Metadata node bound");
+		}
+	}
 
-        private final NCApplicationEntryPoint ncAppEntryPoint;
+	/**
+	 * Shutdown hook that invokes {@link NCApplicationEntryPoint#stop() stop}
+	 * method.
+	 */
+	private static class JVMShutdownHook extends Thread {
 
-        public JVMShutdownHook(NCApplicationEntryPoint ncAppEntryPoint) {
-            this.ncAppEntryPoint = ncAppEntryPoint;
-        }
+		private final NCApplicationEntryPoint ncAppEntryPoint;
 
-        public void run() {
-            if (LOGGER.isLoggable(Level.INFO)) {
-                LOGGER.info("Shutdown hook in progress");
-            }
-            try {
-                ncAppEntryPoint.stop();
-            } catch (Exception e) {
-                if (LOGGER.isLoggable(Level.WARNING)) {
-                    LOGGER.warning("Exception in executing shutdown hook" + e);
-                }
-            }
-        }
-    }
+		public JVMShutdownHook(NCApplicationEntryPoint ncAppEntryPoint) {
+			this.ncAppEntryPoint = ncAppEntryPoint;
+		}
+
+		public void run() {
+			if (LOGGER.isLoggable(Level.INFO)) {
+				LOGGER.info("Shutdown hook in progress");
+			}
+			try {
+				ncAppEntryPoint.stop();
+			} catch (Exception e) {
+				if (LOGGER.isLoggable(Level.WARNING)) {
+					LOGGER.warning("Exception in executing shutdown hook" + e);
+				}
+			}
+		}
+	}
 
 }
\ No newline at end of file
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
index d3c338d..66c0102 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/driver/EventDriver.java
@@ -41,7 +41,7 @@
 public class EventDriver {
 
     public static final String CLIENT_NODE_ID = "client_node";
-    public static final Node CLIENT_NODE = new Node(CLIENT_NODE_ID, "127.0.0.1", null, null, null, null, null);
+    public static final Node CLIENT_NODE = new Node(CLIENT_NODE_ID, "127.0.0.1", null, null, null, null, null, null);
 
     private static String eventsDir;
     private static Events events;
@@ -141,9 +141,9 @@
         args.add(scriptDirSuffix);
         Node clientNode = new Node();
         clientNode.setId("client");
-        clientNode.setIp("127.0.0.1");
+        clientNode.setClusterIp("127.0.0.1");
         for (Node node : cluster.getNode()) {
-            args.add(node.getIp());
+            args.add(node.getClusterIp());
         }
         EventUtil.executeLocalScript(clientNode, eventsDir + "/" + "events" + "/" + "prepare.sh", args);
     }
@@ -153,11 +153,10 @@
         args.add(scriptDirSuffix);
         Node clientNode = new Node();
         clientNode.setId("client");
-        clientNode.setIp("127.0.0.1");
+        clientNode.setClusterIp("127.0.0.1");
         for (Node node : cluster.getNode()) {
-            args.add(node.getIp());
+            args.add(node.getClusterIp());
         }
         EventUtil.executeLocalScript(clientNode, eventsDir + "/" + "events" + "/" + "cleanup.sh", args);
     }
-
 }
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
index 0157f65..6d89c88 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventExecutor.java
@@ -18,6 +18,7 @@
 import java.io.File;
 import java.io.IOException;
 import java.io.StringWriter;
+import java.math.BigInteger;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -44,21 +45,28 @@
         List<String> pargs = new ArrayList<String>();
         pargs.add("/bin/bash");
         pargs.add(client.getEventsDir() + File.separator + "scripts" + File.separator + EXECUTE_SCRIPT);
-        StringBuffer envBuffer = new StringBuffer(IP_LOCATION + "=" + node.getIp() + " ");
+        StringBuffer envBuffer = new StringBuffer(IP_LOCATION + "=" + node.getClusterIp() + " ");
         if (!node.getId().equals(EventDriver.CLIENT_NODE_ID) && cluster.getEnv() != null) {
             for (Property p : cluster.getEnv().getProperty()) {
                 if (p.getKey().equals("JAVA_HOME")) {
                     String val = node.getJavaHome() == null ? p.getValue() : node.getJavaHome();
                     envBuffer.append(p.getKey() + "=" + val + " ");
                 } else if (p.getKey().equals("JAVA_OPTS")) {
-                    String val = "\"" + "-Xmx"
-                            + (node.getJavaHeap() == null ? cluster.getJavaHeap() : node.getJavaHeap());
-                    if (node.getDebug() != null) {
-                        val = val + " " + "-Xdebug -Xrunjdwp:transport=dt_socket,address=" + node.getDebug().intValue()
-                                + "," + "server=y,suspend=n";
+                    StringBuilder builder = new StringBuilder();
+                    builder.append("\"");
+                    String javaOpts = (node.getJavaOpts() == null ? cluster.getJavaOpts() : node.getJavaOpts());
+                    if (javaOpts != null) {
+                        builder.append(javaOpts);
                     }
-                    val = val + "\"";
-                    envBuffer.append(p.getKey() + "=" + val + " ");
+                    if (cluster.isDebugEnabled() != null && cluster.isDebugEnabled().booleanValue()) {
+                        BigInteger debugPort = node.getDebug() == null ? cluster.getDebug() : node.getDebug();
+                        if (debugPort != null) {
+                            builder.append("-Xdebug -Xrunjdwp:transport=dt_socket,address=" + debugPort.intValue()
+                                    + "," + "server=y,suspend=n");
+                        }
+                    }
+                    builder.append("\"");
+                    envBuffer.append(p.getKey() + "=" + builder + " ");
                 } else {
                     envBuffer.append(p.getKey() + "=" + p.getValue() + " ");
                 }
@@ -66,6 +74,7 @@
             }
             pargs.add(cluster.getUsername() == null ? System.getProperty("user.name") : cluster.getUsername());
         }
+
         StringBuffer argBuffer = new StringBuffer();
         if (args != null && args.size() > 0) {
             for (String arg : args) {
@@ -74,7 +83,7 @@
         }
 
         ProcessBuilder pb = new ProcessBuilder(pargs);
-        pb.environment().put(IP_LOCATION, node.getIp());
+        pb.environment().put(IP_LOCATION, node.getClusterIp());
         pb.environment().put(CLUSTER_ENV, envBuffer.toString());
         pb.environment().put(SCRIPT, script);
         pb.environment().put(ARGS, argBuffer.toString());
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
index bf31e9f..e6c58eb 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventUtil.java
@@ -181,16 +181,16 @@
 		}
 
 		if (nodeid.equals(cluster.getMasterNode().getId())) {
-			String ram = cluster.getMasterNode().getJavaHeap() == null ? cluster
-					.getJavaHeap() : cluster.getMasterNode().getJavaHeap();
+			String javaOpts = cluster.getMasterNode().getJavaOpts() == null ? cluster
+					.getJavaOpts() : cluster.getMasterNode().getJavaOpts();
 			String logDir = cluster.getMasterNode().getLogdir() == null ? cluster
 					.getLogdir() : cluster.getMasterNode().getLogdir();
 			String javaHome = cluster.getMasterNode().getJavaHome() == null ? cluster
 					.getJavaHome() : cluster.getMasterNode().getJavaHome();
 			BigInteger debug = cluster.getMasterNode().getDebug();
 			return new Node(cluster.getMasterNode().getId(), cluster
-					.getMasterNode().getIp(), ram, javaHome, logDir, null,
-					debug);
+					.getMasterNode().getClusterIp(), javaHome, javaOpts,
+					logDir, null, null, debug);
 		}
 
 		List<Node> nodeList = cluster.getNode();
@@ -218,7 +218,7 @@
 		pargs.add(EventDriver.getEventsDir() + "/" + EXECUTE_SCRIPT);
 		StringBuffer argBuffer = new StringBuffer();
 		String env = EventDriver.getStringifiedEnv(cluster) + " " + IP_LOCATION
-				+ "=" + node.getIp();
+				+ "=" + node.getClusterIp();
 		if (args != null) {
 			for (String arg : args) {
 				argBuffer.append(arg + " ");
@@ -226,7 +226,7 @@
 		}
 		ProcessBuilder pb = new ProcessBuilder(pargs);
 		pb.environment().putAll(EventDriver.getEnvironment());
-		pb.environment().put(IP_LOCATION, node.getIp());
+		pb.environment().put(IP_LOCATION, node.getClusterIp());
 		pb.environment().put(CLUSTER_ENV, env);
 		pb.environment().put(SCRIPT, script);
 		pb.environment().put(ARGS, argBuffer.toString());
@@ -243,7 +243,7 @@
 		}
 		ProcessBuilder pb = new ProcessBuilder(pargs);
 		pb.environment().putAll(EventDriver.getEnvironment());
-		pb.environment().put(IP_LOCATION, node.getIp());
+		pb.environment().put(IP_LOCATION, node.getClusterIp());
 		pb.start();
 	}
 
diff --git a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
index 4cd4b82..6f4336d 100644
--- a/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
+++ b/asterix-events/src/main/java/edu/uci/ics/asterix/event/management/EventrixClient.java
@@ -38,126 +38,123 @@
 
 public class EventrixClient {
 
-	private static final Logger LOGGER = Logger.getLogger(EventrixClient.class
-			.getName());
+    private static final Logger LOGGER = Logger.getLogger(EventrixClient.class.getName());
 
-	private EventTask[] tasks;
-	private boolean dryRun = false;
-	private LinkedBlockingQueue<EventTaskReport> msgInbox = new LinkedBlockingQueue<EventTaskReport>();
-	private AtomicInteger pendingTasks = new AtomicInteger(0);
-	private final Cluster cluster;
-	private IPatternListener listener;
-	private IOutputHandler outputHandler;
-	private Events events;
-	private String eventsDir;
+    private EventTask[] tasks;
+    private boolean dryRun = false;
+    private LinkedBlockingQueue<EventTaskReport> msgInbox = new LinkedBlockingQueue<EventTaskReport>();
+    private AtomicInteger pendingTasks = new AtomicInteger(0);
+    private final Cluster cluster;
+    private IPatternListener listener;
+    private IOutputHandler outputHandler;
+    private Events events;
+    private String eventsDir;
 
-	public EventrixClient(String eventsDir, Cluster cluster, boolean dryRun,
-			IOutputHandler outputHandler) throws Exception {
-		this.eventsDir = eventsDir;
-		this.events = initializeEvents();
-		this.cluster = cluster;
-		this.dryRun = dryRun;
-		this.outputHandler = outputHandler;
-		if (!dryRun) {
-			initializeCluster(eventsDir);
-		}
-	}
+    public EventrixClient(String eventsDir, Cluster cluster, boolean dryRun, IOutputHandler outputHandler)
+            throws Exception {
+        this.eventsDir = eventsDir;
+        this.events = initializeEvents();
+        this.cluster = cluster;
+        this.dryRun = dryRun;
+        this.outputHandler = outputHandler;
+        if (!dryRun) {
+            initializeCluster(eventsDir);
+        }
+    }
 
-	public void submit(Patterns patterns) throws Exception {
-		initTasks(patterns);
-		try {
-			waitForCompletion();
-		} catch (InterruptedException ie) {
-			LOGGER.info("Interrupted exception :" + ie);
-		} catch (Exception e) {
-			throw e;
-		}
+    public void submit(Patterns patterns) throws Exception {
+        initTasks(patterns);
+        try {
+            waitForCompletion();
+        } catch (InterruptedException ie) {
+            LOGGER.info("Interrupted exception :" + ie);
+        } catch (Exception e) {
+            throw e;
+        }
 
-	}
+    }
 
-	public void submit(Patterns patterns, IPatternListener listener)
-			throws Exception {
-		this.listener = listener;
-		initTasks(patterns);
-	}
+    public void submit(Patterns patterns, IPatternListener listener) throws Exception {
+        this.listener = listener;
+        initTasks(patterns);
+    }
 
-	private void initTasks(Patterns patterns) {
-		tasks = new EventTask[patterns.getPattern().size()];
-		pendingTasks.set(tasks.length);
-		int index = 0;
-		for (Pattern pattern : patterns.getPattern()) {
-			tasks[index] = new EventTask(pattern, this);
-			tasks[index].start();
-			index++;
-		}
-	}
+    private void initTasks(Patterns patterns) {
+        tasks = new EventTask[patterns.getPattern().size()];
+        pendingTasks.set(tasks.length);
+        int index = 0;
+        for (Pattern pattern : patterns.getPattern()) {
+            tasks[index] = new EventTask(pattern, this);
+            tasks[index].start();
+            index++;
+        }
+    }
 
-	public Cluster getCluster() {
-		return cluster;
-	}
+    public Cluster getCluster() {
+        return cluster;
+    }
 
-	public boolean isDryRun() {
-		return dryRun;
-	}
+    public boolean isDryRun() {
+        return dryRun;
+    }
 
-	public Events getEvents() {
-		return events;
-	}
+    public Events getEvents() {
+        return events;
+    }
 
-	public String getEventsDir() {
-		return eventsDir;
-	}
+    public String getEventsDir() {
+        return eventsDir;
+    }
 
-	public synchronized void notifyCompletion(EventTaskReport report) {
+    public synchronized void notifyCompletion(EventTaskReport report) {
 
-		if (report.isSuccess()) {
-			if (listener != null) {
-				pendingTasks.decrementAndGet();
-				listener.eventCompleted(report);
-				if (pendingTasks.get() == 0) {
-					listener.jobCompleted();
-				}
-			} else {
-				try {
-					msgInbox.put(report);
-				} catch (InterruptedException e) {
-				}
-			}
-		} else {
-			for (EventTask t : tasks) {
-				if (t.getState() == EventTask.State.INITIALIZED
-						|| t.getState() == EventTask.State.IN_PROGRESS) {
-					t.cancel();
-				}
-			}
-			if (listener != null) {
-				listener.jobFailed(report);
-			} else {
-				try {
-					msgInbox.put(report);
-				} catch (InterruptedException e) {
-				}
-			}
-		}
-	}
+        if (report.isSuccess()) {
+            if (listener != null) {
+                pendingTasks.decrementAndGet();
+                listener.eventCompleted(report);
+                if (pendingTasks.get() == 0) {
+                    listener.jobCompleted();
+                }
+            } else {
+                try {
+                    msgInbox.put(report);
+                } catch (InterruptedException e) {
+                }
+            }
+        } else {
+            for (EventTask t : tasks) {
+                if (t.getState() == EventTask.State.INITIALIZED || t.getState() == EventTask.State.IN_PROGRESS) {
+                    t.cancel();
+                }
+            }
+            if (listener != null) {
+                listener.jobFailed(report);
+            } else {
+                try {
+                    msgInbox.put(report);
+                } catch (InterruptedException e) {
+                }
+            }
+        }
+    }
 
-	private void waitForCompletion() throws Exception {
-		while (true) {
-			EventTaskReport report = msgInbox.take();
-			if (report.isSuccess()) {
-				if (pendingTasks.decrementAndGet() == 0) {
-					break;
-				}
-			} else {
-				throw new RuntimeException(report.getException().getMessage());
-			}
-		}
-	}
+    private void waitForCompletion() throws Exception {
+        while (true) {
+            EventTaskReport report = msgInbox.take();
+            if (report.isSuccess()) {
+                if (pendingTasks.decrementAndGet() == 0) {
+                    break;
+                }
+            } else {
+                throw new RuntimeException(report.getException().getMessage());
+            }
+        }
+    }
 
-	private void initializeCluster(String eventsDir) throws Exception {
-		Patterns patterns = initPattern(eventsDir);
-		submit(patterns);
-	}
+    private void initializeCluster(String eventsDir) throws Exception {
+        Patterns patterns = initPattern(eventsDir);
+        submit(patterns);
+    }
 
 	private Patterns initPattern(String eventsDir) {
 		Nodeid nodeid = new Nodeid(new Value(null,
@@ -167,37 +164,35 @@
 		String username = cluster.getUsername() == null ? System
 				.getProperty("user.name") : cluster.getUsername();
 		patternList.add(getDirectoryTransferPattern(username, eventsDir,
-				nodeid, cluster.getMasterNode().getIp(), workingDir));
+				nodeid, cluster.getMasterNode().getClusterIp(), workingDir));
 
 		if (!cluster.getWorkingDir().isNFS()) {
 			for (Node node : cluster.getNode()) {
 				patternList.add(getDirectoryTransferPattern(username,
-						eventsDir, nodeid, node.getIp(), workingDir));
+						eventsDir, nodeid, node.getClusterIp(), workingDir));
 			}
 		}
 		Patterns patterns = new Patterns(patternList);
 		return patterns;
 	}
 
-	private Pattern getDirectoryTransferPattern(String username, String src,
-			Nodeid srcNode, String destNodeIp, String destDir) {
-		String pargs = username + " " + src + " " + destNodeIp + " " + destDir;
-		Event event = new Event("directory_transfer", srcNode, pargs);
-		return new Pattern(null, 1, null, event);
-	}
+    private Pattern getDirectoryTransferPattern(String username, String src, Nodeid srcNode, String destNodeIp,
+            String destDir) {
+        String pargs = username + " " + src + " " + destNodeIp + " " + destDir;
+        Event event = new Event("directory_transfer", srcNode, pargs);
+        return new Pattern(null, 1, null, event);
+    }
 
-	public IOutputHandler getErrorHandler() {
-		return outputHandler;
-	}
+    public IOutputHandler getErrorHandler() {
+        return outputHandler;
+    }
 
-	private Events initializeEvents() throws JAXBException,
-			FileNotFoundException {
-		File file = new File(eventsDir + File.separator + "events"
-				+ File.separator + "events.xml");
-		JAXBContext eventCtx = JAXBContext.newInstance(Events.class);
-		Unmarshaller unmarshaller = eventCtx.createUnmarshaller();
-		events = (Events) unmarshaller.unmarshal(file);
-		return events;
-	}
+    private Events initializeEvents() throws JAXBException, FileNotFoundException {
+        File file = new File(eventsDir + File.separator + "events" + File.separator + "events.xml");
+        JAXBContext eventCtx = JAXBContext.newInstance(Events.class);
+        Unmarshaller unmarshaller = eventCtx.createUnmarshaller();
+        events = (Events) unmarshaller.unmarshal(file);
+        return events;
+    }
 
 }
diff --git a/asterix-events/src/main/resources/events/backup/backup.sh b/asterix-events/src/main/resources/events/backup/backup.sh
index e10a4c9..601b81d 100755
--- a/asterix-events/src/main/resources/events/backup/backup.sh
+++ b/asterix-events/src/main/resources/events/backup/backup.sh
@@ -1,35 +1,50 @@
+echo $@ >> ~/backup.log
 WORKING_DIR=$1
 ASTERIX_INSTANCE_NAME=$2
-ASTERIX_DATA_DIR=$3
-BACKUP_ID=$4
-BACKUP_DIR=$5
-BACKUP_TYPE=$6
-NODE_ID=$7
+ASTERIX_IODEVICES=$3
+NODE_STORE=$4
+ASTERIX_ROOT_METADATA_DIR=$5
+BACKUP_ID=$6
+BACKUP_DIR=$7
+BACKUP_TYPE=$8
+NODE_ID=$9
 
-nodeStores=$(echo $ASTERIX_DATA_DIR | tr "," "\n")
+nodeIODevices=$(echo $ASTERIX_IODEVICES | tr "," "\n")
 
 if [ $BACKUP_TYPE == "hdfs" ];
 then
-  HDFS_URL=$8
-  HADOOP_VERSION=$9
+  HDFS_URL=${10}
+  HADOOP_VERSION=${11}
   export HADOOP_HOME=$WORKING_DIR/hadoop-$HADOOP_VERSION
-  for nodeStore in $nodeStores
+  index=1
+  for nodeIODevice in $nodeIODevices
   do
-    MANGLED_DIR_NAME=`echo $nodeStores | tr / _`
+    STORE_DIR=$nodeIODevice/$NODE_STORE
+    MANGLED_DIR_NAME=`echo $STORE_DIR | tr / _`
     NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$MANGLED_DIR_NAME
-    echo "$HADOOP_HOME/bin/hadoop fs -copyFromLocal $nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME/ $HDFS_URL/$NODE_BACKUP_DIR/" >> ~/backup.log
-    $HADOOP_HOME/bin/hadoop fs -copyFromLocal $nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME/ $HDFS_URL/$NODE_BACKUP_DIR/
+    $HADOOP_HOME/bin/hadoop fs -copyFromLocal $STORE_DIR/ $HDFS_URL/$NODE_BACKUP_DIR/
+    if [ $index -eq 1 ];
+    then
+      $HADOOP_HOME/bin/hadoop fs -copyFromLocal $nodeIODevice/$ASTERIX_ROOT_METADATA_DIR $HDFS_URL/$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/
+    fi
+    index=`expr $index + 1`
   done
 else 
-  for nodeStore in $nodeStores
+  index=1
+  for nodeIODevice in $nodeIODevices
   do
-    MANGLED_DIR_NAME=`echo $nodeStores | tr / _`
+    STORE_DIR=$nodeIODevice/$NODE_STORE
+    MANGLED_DIR_NAME=`echo $STORE_DIR | tr / _`
     NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$MANGLED_DIR_NAME
     if [ ! -d $NODE_BACKUP_DIR ];
     then
       mkdir -p $NODE_BACKUP_DIR
     fi
-    echo "cp -r  $nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME/* $NODE_BACKUP_DIR/" >> ~/backup.log
-    cp -r  $nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME/* $NODE_BACKUP_DIR/
+    cp -r  $STORE_DIR/* $NODE_BACKUP_DIR/
+    if [ $index -eq 1 ];
+    then
+      cp -r $nodeIODevice/$ASTERIX_ROOT_METADATA_DIR $BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/
+    fi
+    index=`expr $index + 1`
   done
 fi
diff --git a/asterix-events/src/main/resources/events/node_join/nc_join.sh b/asterix-events/src/main/resources/events/node_join/nc_join.sh
index 5aa86db..6b5280b 100755
--- a/asterix-events/src/main/resources/events/node_join/nc_join.sh
+++ b/asterix-events/src/main/resources/events/node_join/nc_join.sh
@@ -1,7 +1,8 @@
 CC_HOST=$1
 NC_ID=$2
+IO_DEVICES=$3
 if [ ! -d $LOG_DIR ]; 
 then 
   mkdir -p $LOG_DIR
 fi
-$ASTERIX_HOME/bin/asterixnc -node-id $NC_ID -cc-host $CC_HOST -cc-port 1099 -cluster-net-ip-address $IP_LOCATION  -data-ip-address $IP_LOCATION -result-ip-address $IP_LOCATION &> $LOG_DIR/${NC_ID}.log
+$ASTERIX_HOME/bin/asterixnc -node-id $NC_ID -cc-host $CC_HOST -cc-port 1099 -cluster-net-ip-address $IP_LOCATION  -data-ip-address $IP_LOCATION -iodevices $IO_DEVICES -result-ip-address $IP_LOCATION &> $LOG_DIR/${NC_ID}.log
diff --git a/asterix-events/src/main/resources/events/restore/restore.sh b/asterix-events/src/main/resources/events/restore/restore.sh
index 515efc1..85e3eb3 100755
--- a/asterix-events/src/main/resources/events/restore/restore.sh
+++ b/asterix-events/src/main/resources/events/restore/restore.sh
@@ -1,32 +1,53 @@
 WORKING_DIR=$1
 ASTERIX_INSTANCE_NAME=$2
-ASTERIX_DATA_DIR=$3
-BACKUP_ID=$4
-BACKUP_DIR=$5
-BACKUP_TYPE=$6
-NODE_ID=$7
-HDFS_URL=$8
-HADOOP_VERSION=$9
+ASTERIX_IODEVICES=$3
+NODE_STORE=$4
+ASTERIX_ROOT_METADATA_DIR=$5
+BACKUP_ID=$6
+BACKUP_DIR=$7
+BACKUP_TYPE=$8
+NODE_ID=$9
+HDFS_URL=${10}
+HADOOP_VERSION=${11}
 
 export HADOOP_HOME=$WORKING_DIR/hadoop-$HADOOP_VERSION
-nodeStores=$(echo $ASTERIX_DATA_DIR | tr "," "\n")
+iodevices=$(echo $ASTERIX_IODEVICES | tr "," "\n")
 
-for nodeStore in $nodeStores
+index=1
+for iodevice in $iodevices
 do
-  MANGLED_BACKUP_DIR=`echo $nodeStore | tr / _`
+  MANGLED_BACKUP_DIR=`echo $iodevice | tr / _`
   NODE_BACKUP_DIR=$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$MANGLED_BACKUP_DIR
-  DEST_DIR=$nodeStore/$NODE_ID/$ASTERIX_INSTANCE_NAME
+  DEST_DIR=$iodevice/$NODE_STORE/
   if [ ! -d $DEST_DIR ]
   then 
     mkdir -p $DEST_DIR
   else 
-    rm -rf $DEST_DIR/*
+    rm -rf $DEST_DIR
   fi
- 
+
+  if [ ! -d $iodevice/$ASTERIX_ROOT_METADATA_DIR ]
+  then
+    mkdir -p $iodevice/$ASTERIX_ROOT_METADATA_DIR
+  else 
+    rm -rf $iodevice/$ASTERIX_ROOT_METADATA_DIR
+  fi 
+
   if [ $BACKUP_TYPE == "hdfs" ];
   then
     $HADOOP_HOME/bin/hadoop fs -copyToLocal $HDFS_URL/$NODE_BACKUP_DIR/*  $DEST_DIR/ 
+    if [ $index -eq 1 ];
+    then
+      $HADOOP_HOME/bin/hadoop fs -copyToLocal $HDFS_URL/$BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$ASTERIX_ROOT_METADATA_DIR $iodevice/
+    fi
   else
+    echo "cp  -r $NODE_BACKUP_DIR/*  $DEST_DIR/" >> ~/restore.log
     cp  -r $NODE_BACKUP_DIR/*  $DEST_DIR/ 
+    if [ $index -eq 1 ];
+    then
+      echo "cp -r $BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$ASTERIX_ROOT_METADATA_DIR $iodevice/" >> ~/restore.log
+      cp -r $BACKUP_DIR/$ASTERIX_INSTANCE_NAME/$BACKUP_ID/$NODE_ID/$ASTERIX_ROOT_METADATA_DIR $iodevice/
+    fi
   fi
+  index=`expr $index + 1`
 done
diff --git a/asterix-events/src/main/resources/schema/cluster.xsd b/asterix-events/src/main/resources/schema/cluster.xsd
index 38afda3..0e1adce 100644
--- a/asterix-events/src/main/resources/schema/cluster.xsd
+++ b/asterix-events/src/main/resources/schema/cluster.xsd
@@ -3,19 +3,21 @@
 
 <!-- definition of simple types --> 
 <xs:element name="name" type="xs:string"/>
-<xs:element name="java_heap" type="xs:string"/>
+<xs:element name="java_opts" type="xs:string"/>
 <xs:element name="logdir" type="xs:string"/>
 <xs:element name="id" type="xs:string"/>
-<xs:element name="ip" type="xs:string"/>
+<xs:element name="client-ip" type="xs:string"/>
 <xs:element name="cluster-ip" type="xs:string"/>
 <xs:element name="key" type="xs:string"/>
 <xs:element name="value" type="xs:string"/>
 <xs:element name="dir" type="xs:string"/>
 <xs:element name="NFS" type="xs:boolean"/>
 <xs:element name="store" type="xs:string"/>
+<xs:element name="iodevices" type="xs:string"/>
 <xs:element name="java_home" type="xs:string"/>
 <xs:element name="username" type="xs:string"/>
 <xs:element name="debug" type="xs:integer"/>
+<xs:element name="debugEnabled" type="xs:boolean"/>
 
 <!-- definition of complex elements -->
 <xs:element name="workingDir">
@@ -31,10 +33,10 @@
   <xs:complexType>
     <xs:sequence>
       <xs:element ref="cl:id"/>
-      <xs:element ref="cl:ip"/>
+      <xs:element ref="cl:client-ip"/>
       <xs:element ref="cl:cluster-ip"/>
       <xs:element ref="cl:java_home" minOccurs="0"/>
-      <xs:element ref="cl:java_heap" minOccurs="0"/>
+      <xs:element ref="cl:java_opts" minOccurs="0"/>
       <xs:element ref="cl:logdir" minOccurs="0"/>
       <xs:element ref="cl:debug" minOccurs="0"/>
     </xs:sequence>
@@ -62,11 +64,12 @@
   <xs:complexType>
     <xs:sequence>
       <xs:element ref="cl:id"/>
-      <xs:element ref="cl:ip"/>
-      <xs:element ref="cl:java_heap" minOccurs="0"/>
+      <xs:element ref="cl:cluster-ip"/>
       <xs:element ref="cl:java_home" minOccurs="0"/>
+      <xs:element ref="cl:java_opts" minOccurs="0"/>
       <xs:element ref="cl:logdir" minOccurs="0"/>
       <xs:element ref="cl:store" minOccurs="0"/>
+      <xs:element ref="cl:iodevices" minOccurs="0"/>
       <xs:element ref="cl:debug" minOccurs="0"/>
     </xs:sequence>
   </xs:complexType>
@@ -78,11 +81,14 @@
       <xs:element ref="cl:name"/>
       <xs:element ref="cl:username"/>
       <xs:element ref="cl:env" minOccurs="0"/>
-      <xs:element ref="cl:java_heap" minOccurs="0"/>
       <xs:element ref="cl:java_home" minOccurs="0"/>
+      <xs:element ref="cl:java_opts" minOccurs="0"/>
       <xs:element ref="cl:logdir" minOccurs="0"/>
       <xs:element ref="cl:store" minOccurs="0"/>
+      <xs:element ref="cl:iodevices" minOccurs="0"/>
       <xs:element ref="cl:workingDir"/>
+      <xs:element ref="cl:debugEnabled" minOccurs="0"/>
+      <xs:element ref="cl:debug" minOccurs="0"/>
       <xs:element ref="cl:master-node"/>
       <xs:element ref="cl:node" maxOccurs="unbounded"/>
     </xs:sequence>
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommand.java
index af7dbb7..66a0697 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommand.java
@@ -28,14 +28,10 @@
     public void execute(String[] args) throws Exception {
         String[] cmdArgs = new String[args.length - 1];
         System.arraycopy(args, 1, cmdArgs, 0, cmdArgs.length);
-        if (cmdArgs.length >= 1 && cmdArgs[0].equals("-help")) {
-            System.out.println(getUsageDescription());
-        } else {
-            config = getCommandConfig();
-            CmdLineParser parser = new CmdLineParser(config);
-            parser.parseArgument(cmdArgs);
-            execCommand();
-        }
+        config = getCommandConfig();
+        CmdLineParser parser = new CmdLineParser(config);
+        parser.parseArgument(cmdArgs);
+        execCommand();
     }
 
     abstract protected void execCommand() throws Exception;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommandConfig.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommandConfig.java
deleted file mode 100644
index f28dfd9..0000000
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AbstractCommandConfig.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package edu.uci.ics.asterix.installer.command;
-
-import org.kohsuke.args4j.Option;
-
-public class AbstractCommandConfig implements CommandConfig {
-
-    @Option(name = "-help", required = false, usage = "Help")
-    public boolean help = false;
-
-    @Override
-    public boolean helpMode() {
-        return help;
-    }
-
-}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
index e52308e..5ef7449 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/AlterCommand.java
@@ -52,14 +52,13 @@
     protected String getUsageDescription() {
         return "\nAlter the instance's configuration settings."
                 + "\nPrior to running this command, the instance is required to be INACTIVE state."
-                + "\n\nAvailable arguments/options" 
-                + "\n-n name of the ASTERIX instance"
+                + "\n\nAvailable arguments/options" + "\n-n name of the ASTERIX instance"
                 + "\n-conf path to the ASTERIX configuration file.";
     }
 
 }
 
-class AlterConfig extends AbstractCommandConfig {
+class AlterConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
     public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
index cd4cc5c..297a168 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/BackupCommand.java
@@ -31,6 +31,8 @@
 
 public class BackupCommand extends AbstractCommand {
 
+    public static final String ASTERIX_ROOT_METADATA_DIR = "asterix_root_metadata";
+
     @Override
     protected void execCommand() throws Exception {
         InstallerDriver.initConfig();
@@ -59,15 +61,14 @@
                 + "\nfailure or accidental execution of a DDL statement (drop dataverse/dataset),"
                 + "\nyou may need to recover the lost data. The backup command allows you to take a"
                 + "\nbackup of the data stored with an ASTERIX instance. "
-                + "\nThe backed up snapshot is stored in HDFS." 
-                + "\n\nAvailable arguments/options:"
+                + "\nThe backed up snapshot is stored in HDFS." + "\n\nAvailable arguments/options:"
                 + "\n-n name of the Asterix instance";
 
     }
 
 }
 
-class BackupConfig extends AbstractCommandConfig {
+class BackupConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of the Asterix instance")
     public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandConfig.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandConfig.java
index 94aa935..55b0c3b 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandConfig.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandConfig.java
@@ -1,20 +1,5 @@
-/*
- * Copyright 2009-2012 by The Regents of the University of California
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * you may obtain a copy of the License from
- * 
- *     http://www.apache.org/licenses/LICENSE-2.0
- * 
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
 package edu.uci.ics.asterix.installer.command;
 
-public interface CommandConfig {
+public abstract class CommandConfig {
 
-    public boolean helpMode();
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
index 8a9347a..7fc3884 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CommandHandler.java
@@ -55,6 +55,9 @@
             case SHUTDOWN:
                 cmd = new ShutdownCommand();
                 break;
+            case HELP:
+                cmd = new HelpCommand();
+                break;
         }
         cmd.execute(args);
     }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ConfigureCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ConfigureCommand.java
index 6b0b53d..cad8632 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ConfigureCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ConfigureCommand.java
@@ -26,9 +26,11 @@
         String workingDir = InstallerDriver.getManagixHome() + File.separator + "clusters" + File.separator + "local"
                 + File.separator + "working_dir";
         cluster.setWorkingDir(new WorkingDir(workingDir, true));
-        cluster.setStore(workingDir + File.separator + "storage");
+        cluster.setIodevices(workingDir);
+        cluster.setStore("storage");
         cluster.setLogdir(workingDir + File.separator + "logs");
         cluster.setJavaHome(System.getenv("JAVA_HOME"));
+        cluster.setJavaOpts("-Xmx1048m");
 
         Marshaller marshaller = ctx.createMarshaller();
         marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true);
@@ -50,7 +52,7 @@
     @Override
     protected String getUsageDescription() {
         return "\nAuto-generates the ASTERIX installer configruation settings and ASTERIX cluster "
-                + "\n configuration settings for a single node setup.";
+                + "\nconfiguration settings for a single node setup.";
     }
 
     @Override
@@ -60,6 +62,6 @@
 
 }
 
-class ConfigureConfig extends AbstractCommandConfig {
+class ConfigureConfig extends CommandConfig {
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
index 5ddc9e9..fd0af04 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/CreateCommand.java
@@ -57,11 +57,6 @@
 		Unmarshaller unmarshaller = ctx.createUnmarshaller();
 		cluster = (Cluster) unmarshaller.unmarshal(new File(
 				createConfig.clusterPath));
-		cluster.setWorkingDir(new WorkingDir(cluster.getWorkingDir().getDir()
-				+ File.separator + asterixInstanceName, cluster.getWorkingDir()
-				.isNFS()));
-		cluster.setLogdir(cluster.getLogdir() + File.separator
-				+ asterixInstanceName);
 		AsterixInstance asterixInstance = InstallerUtil.createAsterixInstance(
 				asterixInstanceName, cluster);
 		InstallerUtil.evaluateConflictWithOtherInstances(asterixInstance);
@@ -69,12 +64,15 @@
 		List<Property> clusterProperties = new ArrayList<Property>();
 		clusterProperties.add(new Property("ASTERIX_HOME", cluster
 				.getWorkingDir().getDir() + File.separator + "asterix"));
-		clusterProperties.add(new Property("JAVA_OPTS", "-Xmx"
-				+ cluster.getJavaHeap()));
+		StringBuilder javaOpts = new StringBuilder();
+		if (cluster.getJavaOpts() != null) {
+			javaOpts.append(cluster.getJavaOpts());
+		}
+		clusterProperties.add(new Property("JAVA_OPTS", javaOpts.toString()));
 		clusterProperties.add(new Property("CLUSTER_NET_IP", cluster
 				.getMasterNode().getClusterIp()));
 		clusterProperties.add(new Property("CLIENT_NET_IP", cluster
-				.getMasterNode().getIp()));
+				.getMasterNode().getClientIp()));
 		clusterProperties.add(new Property("LOG_DIR", cluster.getLogdir()));
 		clusterProperties.add(new Property("JAVA_HOME", cluster.getJavaHome()));
 		clusterProperties.add(new Property("WORKING_DIR", cluster
@@ -123,7 +121,7 @@
 
 }
 
-class CreateConfig extends AbstractCommandConfig {
+class CreateConfig extends CommandConfig {
 
 	@Option(name = "-n", required = true, usage = "Name of Asterix Instance")
 	public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
index a2788b4..2279d8f 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DeleteCommand.java
@@ -26,41 +26,37 @@
 
 public class DeleteCommand extends AbstractCommand {
 
-	@Override
-	protected void execCommand() throws Exception {
-		InstallerDriver.initConfig();
-		String asterixInstanceName = ((DeleteConfig) config).name;
-		AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(
-				asterixInstanceName, State.INACTIVE);
-		PatternCreator pc = new PatternCreator();
-		Patterns patterns = pc.createDeleteInstancePattern(instance);
-		InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig();
+        String asterixInstanceName = ((DeleteConfig) config).name;
+        AsterixInstance instance = InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE);
+        PatternCreator pc = new PatternCreator();
+        Patterns patterns = pc.createDeleteInstancePattern(instance);
+        InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
 
-		patterns = pc.createRemoveAsterixWorkingDirPattern(instance);
-		InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
-		ServiceProvider.INSTANCE.getLookupService().removeAsterixInstance(
-				asterixInstanceName);
-		LOGGER.info("Deleted Asterix instance: " + asterixInstanceName);
-	}
+        patterns = pc.createRemoveAsterixWorkingDirPattern(instance);
+        InstallerUtil.getEventrixClient(instance.getCluster()).submit(patterns);
+        ServiceProvider.INSTANCE.getLookupService().removeAsterixInstance(asterixInstanceName);
+        LOGGER.info("Deleted Asterix instance: " + asterixInstanceName);
+    }
 
-	@Override
-	protected CommandConfig getCommandConfig() {
-		return new DeleteConfig();
-	}
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new DeleteConfig();
+    }
 
-	@Override
-	protected String getUsageDescription() {
-		return "\nPermanently deletes an ASTERIX instance." + "\n"
-				+ "The instance must be in the INACTIVE state."
-				+ "\n\nAvailable arguments/options"
-				+ "\n-n name of the ASTERIX instance.";
-	}
+    @Override
+    protected String getUsageDescription() {
+        return "\nPermanently deletes an ASTERIX instance." + "\n" + "The instance must be in the INACTIVE state."
+                + "\n\nAvailable arguments/options" + "\n-n name of the ASTERIX instance.";
+    }
 
 }
 
-class DeleteConfig extends AbstractCommandConfig {
+class DeleteConfig extends CommandConfig {
 
-	@Option(name = "-n", required = true, usage = "Name of Asterix Instance")
-	public String name;
+    @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
+    public String name;
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
index bec82d5..9306a56 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/DescribeCommand.java
@@ -29,77 +29,63 @@
 
 public class DescribeCommand extends AbstractCommand {
 
-	@Override
-	protected void execCommand() throws Exception {
-		InstallerDriver.initConfig();
-		String asterixInstanceName = ((DescribeConfig) config).name;
-		boolean adminView = ((DescribeConfig) config).admin;
-		if (asterixInstanceName != null) {
-			InstallerUtil.validateAsterixInstanceExists(asterixInstanceName,
-					State.INACTIVE, State.ACTIVE, State.UNUSABLE);
-			AsterixInstance instance = ServiceProvider.INSTANCE
-					.getLookupService().getAsterixInstance(asterixInstanceName);
-			if (instance != null) {
-				AsterixRuntimeState state = VerificationUtil
-						.getAsterixRuntimeState(instance);
-				boolean expectedRunning = instance.getState().equals(
-						State.UNUSABLE) ? instance.getPreviousState().equals(
-						State.ACTIVE) : !instance.getState().equals(
-						State.INACTIVE);
-				VerificationUtil.updateInstanceWithRuntimeDescription(instance,
-						state, expectedRunning);
-				ServiceProvider.INSTANCE.getLookupService()
-						.updateAsterixInstance(instance);
-				LOGGER.info(instance.getDescription(adminView));
-			} else {
-				throw new InstallerException("Asterix instance by the name "
-						+ asterixInstanceName + " does not exist.");
-			}
-		} else {
-			List<AsterixInstance> asterixInstances = ServiceProvider.INSTANCE
-					.getLookupService().getAsterixInstances();
-			if (asterixInstances.size() > 0) {
-				for (AsterixInstance instance : asterixInstances) {
-					AsterixRuntimeState state = VerificationUtil
-							.getAsterixRuntimeState(instance);
-					boolean expectedRunning = instance.getState().equals(
-							State.UNUSABLE) ? instance.getPreviousState()
-							.equals(State.ACTIVE) : !instance.getState()
-							.equals(State.INACTIVE);
-					VerificationUtil.updateInstanceWithRuntimeDescription(
-							instance, state, expectedRunning);
-					ServiceProvider.INSTANCE.getLookupService()
-							.updateAsterixInstance(instance);
-					LOGGER.info(instance.getDescription(adminView));
-				}
-			} else {
-				LOGGER.info("No Asterix instances found!");
-			}
+    @Override
+    protected void execCommand() throws Exception {
+        InstallerDriver.initConfig();
+        String asterixInstanceName = ((DescribeConfig) config).name;
+        boolean adminView = ((DescribeConfig) config).admin;
+        if (asterixInstanceName != null) {
+            InstallerUtil.validateAsterixInstanceExists(asterixInstanceName, State.INACTIVE, State.ACTIVE,
+                    State.UNUSABLE);
+            AsterixInstance instance = ServiceProvider.INSTANCE.getLookupService().getAsterixInstance(
+                    asterixInstanceName);
+            if (instance != null) {
+                AsterixRuntimeState state = VerificationUtil.getAsterixRuntimeState(instance);
+                boolean expectedRunning = instance.getState().equals(State.UNUSABLE) ? instance.getPreviousState()
+                        .equals(State.ACTIVE) : !instance.getState().equals(State.INACTIVE);
+                VerificationUtil.updateInstanceWithRuntimeDescription(instance, state, expectedRunning);
+                ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(instance);
+                LOGGER.info(instance.getDescription(adminView));
+            } else {
+                throw new InstallerException("Asterix instance by the name " + asterixInstanceName + " does not exist.");
+            }
+        } else {
+            List<AsterixInstance> asterixInstances = ServiceProvider.INSTANCE.getLookupService().getAsterixInstances();
+            if (asterixInstances.size() > 0) {
+                for (AsterixInstance instance : asterixInstances) {
+                    AsterixRuntimeState state = VerificationUtil.getAsterixRuntimeState(instance);
+                    boolean expectedRunning = instance.getState().equals(State.UNUSABLE) ? instance.getPreviousState()
+                            .equals(State.ACTIVE) : !instance.getState().equals(State.INACTIVE);
+                    VerificationUtil.updateInstanceWithRuntimeDescription(instance, state, expectedRunning);
+                    ServiceProvider.INSTANCE.getLookupService().updateAsterixInstance(instance);
+                    LOGGER.info(instance.getDescription(adminView));
+                }
+            } else {
+                LOGGER.info("No Asterix instances found!");
+            }
 
-		}
-	}
+        }
+    }
 
-	@Override
-	protected CommandConfig getCommandConfig() {
-		return new DescribeConfig();
-	}
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new DescribeConfig();
+    }
 
-	@Override
-	protected String getUsageDescription() {
-		return "\nProvides information about an ASTERIX instance."
-				+ "\n\nUsage arguments/options:-"
-				+ "\n[-n]       Name of the ASTERIX instance."
-				+ "\n[-admin]   Provides a detailed description";
-	}
+    @Override
+    protected String getUsageDescription() {
+        return "\nProvides information about an ASTERIX instance." + "\n\nUsage arguments/options:-"
+                + "\n[-n]       Name of the ASTERIX instance." + "\n[-admin]   Provides a detailed description";
+    }
 
 }
 
-class DescribeConfig extends AbstractCommandConfig {
+class DescribeConfig extends CommandConfig {
 
-	@Option(name = "-n", required = false, usage = "Name of Asterix Instance")
-	public String name;
+    @Option(name = "-n", required = false, usage = "Name of Asterix Instance")
+    public String name;
 
-	@Option(name = "-admin", required = false, usage = "Detailed description")
-	public boolean admin;
+    @Option(name = "-admin", required = false, usage = "Detailed description")
+    public boolean admin;
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
new file mode 100644
index 0000000..a9864a3
--- /dev/null
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/HelpCommand.java
@@ -0,0 +1,85 @@
+/*
+ * Copyright 2009-2012 by The Regents of the University of California
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * you may obtain a copy of the License from
+ * 
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package edu.uci.ics.asterix.installer.command;
+
+import org.kohsuke.args4j.Option;
+
+public class HelpCommand extends AbstractCommand {
+
+    @Override
+    protected void execCommand() throws Exception {
+        HelpConfig helpConfig = (HelpConfig) config;
+        String command = helpConfig.command;
+        CommandType cmdType = CommandType.valueOf(command.toUpperCase());
+        String helpMessage = null;
+        switch (cmdType) {
+            case CREATE:
+                helpMessage = new CreateCommand().getUsageDescription();
+                break;
+            case CONFIGURE:
+                helpMessage = new ConfigureCommand().getUsageDescription();
+                break;
+            case DELETE:
+                helpMessage = new DeleteCommand().getUsageDescription();
+                break;
+            case DESCRIBE:
+                helpMessage = new DescribeCommand().getUsageDescription();
+                break;
+            case RESTORE:
+                helpMessage = new RestoreCommand().getUsageDescription();
+                break;
+            case START:
+                helpMessage = new StartCommand().getUsageDescription();
+                break;
+            case SHUTDOWN:
+                helpMessage = new ShutdownCommand().getUsageDescription();
+                break;
+            case BACKUP:
+                helpMessage = new BackupCommand().getUsageDescription();
+                break;
+            case STOP:
+                helpMessage = new StopCommand().getUsageDescription();
+                break;
+            case VALIDATE:
+                helpMessage = new ValidateCommand().getUsageDescription();
+                break;
+            default:
+                helpMessage = "Unknown command " + command;
+        }
+
+        System.out.println(helpMessage);
+    }
+
+    @Override
+    protected CommandConfig getCommandConfig() {
+        return new HelpConfig();
+    }
+
+    @Override
+    protected String getUsageDescription() {
+        return "\nAlter the instance's configuration settings."
+                + "\nPrior to running this command, the instance is required to be INACTIVE state."
+                + "\n\nAvailable arguments/options" + "\n-n name of the ASTERIX instance"
+                + "\n-conf path to the ASTERIX configuration file.";
+    }
+
+}
+
+class HelpConfig extends CommandConfig {
+
+    @Option(name = "-cmd", required = true, usage = "Name of Asterix Instance")
+    public String command;
+
+}
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
index d27fee9..979c414 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ICommand.java
@@ -27,7 +27,8 @@
         ALTER,
         VALIDATE,
         CONFIGURE,
-        SHUTDOWN
+        SHUTDOWN,
+        HELP
     }
 
     public void execute(String args[]) throws Exception;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
index 49629a4..9ef925a 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/RestoreCommand.java
@@ -60,7 +60,7 @@
 
 }
 
-class RestoreConfig extends AbstractCommandConfig {
+class RestoreConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of the Asterix instance")
     public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
index cbfd5c5..b9dd23d 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ShutdownCommand.java
@@ -39,6 +39,6 @@
 
 }
 
-class ShutdownConfig extends AbstractCommandConfig {
+class ShutdownConfig extends CommandConfig {
 
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
index 8004e9a..38bcf33 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StartCommand.java
@@ -61,11 +61,9 @@
     }
 }
 
-class StartConfig extends AbstractCommandConfig {
+class StartConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
     public String name;
 
-  
-
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
index 85fcb68..0e10414 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/StopCommand.java
@@ -80,7 +80,7 @@
 
 }
 
-class StopConfig extends AbstractCommandConfig {
+class StopConfig extends CommandConfig {
 
     @Option(name = "-n", required = true, usage = "Name of Asterix Instance")
     public String name;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
index 42c0d38..3d292a9 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/command/ValidateCommand.java
@@ -29,18 +29,15 @@
 import edu.uci.ics.asterix.event.schema.cluster.Cluster;
 import edu.uci.ics.asterix.event.schema.cluster.MasterNode;
 import edu.uci.ics.asterix.event.schema.cluster.Node;
-import edu.uci.ics.asterix.event.schema.pattern.Patterns;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
-import edu.uci.ics.asterix.installer.driver.InstallerUtil;
-import edu.uci.ics.asterix.installer.events.PatternCreator;
 import edu.uci.ics.asterix.installer.schema.conf.Configuration;
 import edu.uci.ics.asterix.installer.schema.conf.Zookeeper;
 
 public class ValidateCommand extends AbstractCommand {
 
-	private static final String OK = " [" + "\u2713" + "]";
-	private static final String ERROR = " [" + "x" + "]";
-	private static final String WARNING = " [" + "!" + "]";
+	private static final String OK = " [" + "OK" + "]";
+	private static final String ERROR = " [" + "ERROR" + "]";
+	private static final String WARNING = " [" + "WARNING" + "]";
 
 	@Override
 	protected void execCommand() throws Exception {
@@ -111,18 +108,19 @@
 
 			Set<String> servers = new HashSet<String>();
 			Set<String> serverIds = new HashSet<String>();
-			servers.add(cluster.getMasterNode().getIp());
+			servers.add(cluster.getMasterNode().getClusterIp());
 			serverIds.add(cluster.getMasterNode().getId());
 
 			MasterNode masterNode = cluster.getMasterNode();
-			Node master = new Node(masterNode.getId(), masterNode.getIp(),
-					masterNode.getJavaHeap(), masterNode.getJavaHome(),
-					masterNode.getLogdir(), null, masterNode.getDebug());
+			Node master = new Node(masterNode.getId(),
+					masterNode.getClusterIp(), masterNode.getJavaOpts(),
+					masterNode.getJavaHome(), masterNode.getLogdir(), null,
+					null, null);
 
 			valid = valid & validateNodeConfiguration(master, cluster);
 
 			for (Node node : cluster.getNode()) {
-				servers.add(node.getIp());
+				servers.add(node.getClusterIp());
 				if (serverIds.contains(node.getId())) {
 					valid = false;
 					LOGGER.error("Duplicate node id :" + node.getId() + ERROR);
@@ -141,9 +139,9 @@
 				&& checkTemporaryPath(cluster.getLogdir())) {
 			tempDirs.add("Log directory: " + cluster.getLogdir());
 		}
-		if (cluster.getStore() != null
-				&& checkTemporaryPath(cluster.getStore())) {
-			tempDirs.add("Store directory: " + cluster.getStore());
+		if (cluster.getIodevices() != null
+				&& checkTemporaryPath(cluster.getIodevices())) {
+			tempDirs.add("IO Device: " + cluster.getIodevices());
 		}
 
 		if (tempDirs.size() > 0) {
@@ -159,7 +157,7 @@
 
 	private boolean validateNodeConfiguration(Node node, Cluster cluster) {
 		boolean valid = true;
-		valid = checkNodeReachability(node.getIp());
+		valid = checkNodeReachability(node.getClusterIp());
 		if (node.getJavaHome() == null || node.getJavaHome().length() == 0) {
 			if (cluster.getJavaHome() == null
 					|| cluster.getJavaHome().length() == 0) {
@@ -178,8 +176,8 @@
 			}
 		}
 
-		if (node.getStore() == null || cluster.getStore().length() == 0) {
-			if (cluster.getMasterNode().getId().equals(node.getId())
+		if (node.getStore() == null || node.getStore().length() == 0) {
+			if (!cluster.getMasterNode().getId().equals(node.getId())
 					&& (cluster.getStore() == null || cluster.getStore()
 							.length() == 0)) {
 				valid = false;
@@ -188,14 +186,16 @@
 			}
 		}
 
-		if (node.getJavaHeap() == null || node.getJavaHeap().length() == 0) {
-			if (cluster.getJavaHeap() == null
-					|| cluster.getJavaHeap().length() == 0) {
+		if (node.getIodevices() == null || node.getIodevices().length() == 0) {
+			if (!cluster.getMasterNode().getId().equals(node.getId())
+					&& (cluster.getIodevices() == null || cluster
+							.getIodevices().length() == 0)) {
 				valid = false;
-				LOGGER.fatal("java heap size not defined at cluster/node level for node: "
+				LOGGER.fatal("iodevice(s) not defined at cluster/node level for node: "
 						+ node.getId() + ERROR);
 			}
 		}
+
 		return valid;
 	}
 
@@ -259,7 +259,7 @@
 
 }
 
-class ValidateConfig extends AbstractCommandConfig {
+class ValidateConfig extends CommandConfig {
 
 	@Option(name = "-c", required = false, usage = "Path to the cluster configuration xml")
 	public String cluster;
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
index 321fd1a..8083427 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerDriver.java
@@ -30,107 +30,133 @@
 
 public class InstallerDriver {
 
-    public static final String MANAGIX_INTERNAL_DIR = ".installer";
-    public static final String MANAGIX_EVENT_DIR = MANAGIX_INTERNAL_DIR + File.separator + "eventrix";
-    public static final String MANAGIX_EVENT_SCRIPTS_DIR = MANAGIX_INTERNAL_DIR + File.separator + "eventrix"
-            + File.separator + "scripts";
-    public static final String ASTERIX_DIR = "asterix";
-    public static final String EVENTS_DIR = "events";
+	public static final String MANAGIX_INTERNAL_DIR = ".installer";
+	public static final String MANAGIX_EVENT_DIR = MANAGIX_INTERNAL_DIR
+			+ File.separator + "eventrix";
+	public static final String MANAGIX_EVENT_SCRIPTS_DIR = MANAGIX_INTERNAL_DIR
+			+ File.separator + "eventrix" + File.separator + "scripts";
+	public static final String ASTERIX_DIR = "asterix";
+	public static final String EVENTS_DIR = "events";
 
-    private static final Logger LOGGER = Logger.getLogger(InstallerDriver.class.getName());
-    public static final String ENV_MANAGIX_HOME = "MANAGIX_HOME";
-    public static final String MANAGIX_CONF_XML = "conf" + File.separator + "managix-conf.xml";
+	private static final Logger LOGGER = Logger.getLogger(InstallerDriver.class
+			.getName());
+	public static final String ENV_MANAGIX_HOME = "MANAGIX_HOME";
+	public static final String MANAGIX_CONF_XML = "conf" + File.separator
+			+ "managix-conf.xml";
 
-    private static Configuration conf;
-    private static String managixHome;
-    private static String asterixZip;
+	private static Configuration conf;
+	private static String managixHome;
+	private static String asterixZip;
 
-    public static String getAsterixZip() {
-        return asterixZip;
-    }
+	public static String getAsterixZip() {
+		return asterixZip;
+	}
 
-    public static Configuration getConfiguration() {
-        return conf;
-    }
+	public static Configuration getConfiguration() {
+		return conf;
+	}
 
-    public static void initConfig() throws Exception {
-        File configFile = new File(managixHome + File.separator + MANAGIX_CONF_XML);
-        JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
-        Unmarshaller unmarshaller = configCtx.createUnmarshaller();
-        conf = (Configuration) unmarshaller.unmarshal(configFile);
-        asterixZip = initBinary("asterix-server");
+	public static void initConfig() throws Exception {
+		File configFile = new File(managixHome + File.separator
+				+ MANAGIX_CONF_XML);
+		JAXBContext configCtx = JAXBContext.newInstance(Configuration.class);
+		Unmarshaller unmarshaller = configCtx.createUnmarshaller();
+		conf = (Configuration) unmarshaller.unmarshal(configFile);
+		asterixZip = initBinary("asterix-server");
 
-        ILookupService lookupService = ServiceProvider.INSTANCE.getLookupService();
-        if (!lookupService.isRunning(conf)) {
-            lookupService.startService(conf);
-        }
-    }
+		ILookupService lookupService = ServiceProvider.INSTANCE
+				.getLookupService();
+		if (!lookupService.isRunning(conf)) {
+			lookupService.startService(conf);
+		}
+	}
 
-    private static String initBinary(final String fileNamePattern) {
-        String asterixDir = InstallerDriver.getAsterixDir();
-        File file = new File(asterixDir);
-        File[] zipFiles = file.listFiles(new FileFilter() {
-            public boolean accept(File arg0) {
-                return arg0.getAbsolutePath().contains(fileNamePattern) && arg0.isFile();
-            }
-        });
-        if (zipFiles.length == 0) {
-            String msg = " Binary not found at " + asterixDir;
-            LOGGER.log(Level.FATAL, msg);
-            throw new IllegalStateException(msg);
-        }
-        if (zipFiles.length > 1) {
-            String msg = " Multiple binaries found at " + asterixDir;
-            LOGGER.log(Level.FATAL, msg);
-            throw new IllegalStateException(msg);
-        }
+	private static String initBinary(final String fileNamePattern) {
+		String asterixDir = InstallerDriver.getAsterixDir();
+		File file = new File(asterixDir);
+		File[] zipFiles = file.listFiles(new FileFilter() {
+			public boolean accept(File arg0) {
+				return arg0.getAbsolutePath().contains(fileNamePattern)
+						&& arg0.isFile();
+			}
+		});
+		if (zipFiles.length == 0) {
+			String msg = " Binary not found at " + asterixDir;
+			LOGGER.log(Level.FATAL, msg);
+			throw new IllegalStateException(msg);
+		}
+		if (zipFiles.length > 1) {
+			String msg = " Multiple binaries found at " + asterixDir;
+			LOGGER.log(Level.FATAL, msg);
+			throw new IllegalStateException(msg);
+		}
 
-        return zipFiles[0].getAbsolutePath();
-    }
+		return zipFiles[0].getAbsolutePath();
+	}
 
-    public static String getManagixHome() {
-        return managixHome;
-    }
+	public static String getManagixHome() {
+		return managixHome;
+	}
 
-    public static String getAsterixDir() {
-        return managixHome + File.separator + ASTERIX_DIR;
-    }
+	public static String getAsterixDir() {
+		return managixHome + File.separator + ASTERIX_DIR;
+	}
 
-    public static void main(String args[]) {
-        try {
-            if (args.length != 0) {
-                managixHome = System.getenv(ENV_MANAGIX_HOME);
-                CommandHandler cmdHandler = new CommandHandler();
-                cmdHandler.processCommand(args);
-            } else {
-                printUsage();
-            }
-        } catch (IllegalArgumentException iae) {
-            LOGGER.error("Unknown command");
-            printUsage();
-        } catch (Exception e) {
-            LOGGER.error(e.getMessage());
-        }
-    }
+	public static void main(String args[]) {
+		try {
+			if (args.length != 0) {
+				managixHome = System.getenv(ENV_MANAGIX_HOME);
+				CommandHandler cmdHandler = new CommandHandler();
+				cmdHandler.processCommand(args);
+			} else {
+				printUsage();
+			}
+		} catch (IllegalArgumentException iae) {
+			LOGGER.error("Unknown command");
+			printUsage();
+		} catch (Exception e) {
+			LOGGER.error(e.getMessage());
+			if (e.getMessage() == null || e.getMessage().length() == 0) {
+				e.printStackTrace();
+			}
+		}
+	}
 
-    private static void printUsage() {
-        StringBuffer buffer = new StringBuffer("managix <command> <options>" + "\n");
-        buffer.append("Commands" + "\n");
-        buffer.append("create   " + ":" + " Creates a new asterix instance" + "\n");
-        buffer.append("delete   " + ":" + " Deletes an asterix instance" + "\n");
-        buffer.append("start    " + ":" + " Starts an  asterix instance" + "\n");
-        buffer.append("stop     " + ":" + " Stops an asterix instance that is in ACTIVE state" + "\n");
-        buffer.append("backup   " + ":" + " Creates a back up for an existing asterix instance" + "\n");
-        buffer.append("restore  " + ":" + " Restores an asterix instance" + "\n");
-        buffer.append("describe " + ":" + " Describes an existing asterix instance" + "\n");
-        buffer.append("validate " + ":" + " Validates the installer/cluster configuration" + "\n");
-        buffer.append("configure" + ":" + " Auto-generate configuration for local psedu-distributed Asterix instance"
-                + "\n");
-        buffer.append("shutdown " + ":" + " Shutdown the installer service" + "\n");
-        buffer.append("validate " + ":" + " Validates the installer/cluster configuration" + "\n");
-        buffer.append("configure" + ":" + " Auto-generate configuration for local psedu-distributed Asterix instance"
-                + "\n");
-        buffer.append("shutdown " + ":" + " Shutdown the installer service" + "\n");
-        LOGGER.info(buffer.toString());
-    }
+	private static void printUsage() {
+		StringBuffer buffer = new StringBuffer("managix <command> <options>"
+				+ "\n");
+		buffer.append("Commands" + "\n");
+		buffer.append("create   " + ":" + " Creates a new asterix instance"
+				+ "\n");
+		buffer.append("delete   " + ":" + " Deletes an asterix instance" + "\n");
+		buffer.append("start    " + ":" + " Starts an  asterix instance" + "\n");
+		buffer.append("stop     " + ":"
+				+ " Stops an asterix instance that is in ACTIVE state" + "\n");
+		buffer.append("backup   " + ":"
+				+ " Creates a back up for an existing asterix instance" + "\n");
+		buffer.append("restore  " + ":" + " Restores an asterix instance"
+				+ "\n");
+		buffer.append("describe " + ":"
+				+ " Describes an existing asterix instance" + "\n");
+		buffer.append("validate " + ":"
+				+ " Validates the installer/cluster configuration" + "\n");
+		buffer.append("configure"
+				+ ":"
+				+ " Auto-generate configuration for local psedu-distributed Asterix instance"
+				+ "\n");
+		buffer.append("shutdown " + ":" + " Shutdown the installer service"
+				+ "\n");
+		buffer.append("validate " + ":"
+				+ " Validates the installer/cluster configuration" + "\n");
+		buffer.append("configure"
+				+ ":"
+				+ " Auto-generate configuration for local psedu-distributed Asterix instance"
+				+ "\n");
+		buffer.append("shutdown " + ":" + " Shutdown the installer service"
+				+ "\n");
+		buffer.append("help     " + ":"
+				+ " Provides usage description of a command" + "\n");
+
+		LOGGER.info(buffer.toString());
+	}
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
index 24b531a..2f118c4 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/driver/InstallerUtil.java
@@ -25,9 +25,12 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.StringWriter;
+import java.net.URL;
+import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.Enumeration;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
 import java.util.Random;
 import java.util.jar.JarEntry;
@@ -65,20 +68,57 @@
         return instance;
     }
 
-    public static void createAsterixZip(AsterixInstance asterixInstance, boolean newDeployment) throws IOException {
+    public static void createAsterixZip(AsterixInstance asterixInstance, boolean newDeployment) throws IOException,
+            InterruptedException {
+
+        String modifiedZipPath = injectAsterixPropertyFile(InstallerDriver.getAsterixZip(), asterixInstance,
+                newDeployment);
+        injectAsterixLogPropertyFile(modifiedZipPath, asterixInstance);
+    }
+
+    private static String injectAsterixPropertyFile(String origZipFile, AsterixInstance asterixInstance,
+            boolean newDeployment) throws IOException {
         writeAsterixConfigurationFile(asterixInstance, newDeployment);
         String asterixInstanceDir = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName();
-        unzip(InstallerDriver.getAsterixZip(), asterixInstanceDir);
+        unzip(origZipFile, asterixInstanceDir);
         File sourceJar = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
                 + asterixInstance.getAsterixVersion() + ".jar");
-        String origFile = "test.properties";
+        String asterixPropertyFile = "test.properties";
         File replacementFile = new File(asterixInstanceDir + File.separator + "test.properties");
-        replaceInJar(sourceJar, origFile, replacementFile);
+        replaceInJar(sourceJar, asterixPropertyFile, replacementFile);
         new File(asterixInstanceDir + File.separator + "test.properties").delete();
         String asterixZipName = InstallerDriver.getAsterixZip().substring(
                 InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
         zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+        return asterixInstanceDir + File.separator + asterixZipName;
+    }
 
+    private static String injectAsterixLogPropertyFile(String origZipFile, AsterixInstance asterixInstance)
+            throws IOException {
+        String asterixInstanceDir = InstallerDriver.getAsterixDir() + File.separator + asterixInstance.getName();
+        unzip(origZipFile, asterixInstanceDir);
+        File sourceJar1 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+                + asterixInstance.getAsterixVersion() + ".jar");
+        String txnLogPropertyFile = "log.properties";
+        Properties txnLogProperties = new Properties();
+        URLClassLoader urlClassLoader = new URLClassLoader(new URL[] { sourceJar1.toURI().toURL() });
+        InputStream in = urlClassLoader.getResourceAsStream(txnLogPropertyFile);
+        if (in != null) {
+            txnLogProperties.load(in);
+        }
+
+        writeAsterixLogConfigurationFile(asterixInstance.getName(), asterixInstance.getCluster(), txnLogProperties);
+
+        File sourceJar2 = new File(asterixInstanceDir + File.separator + "lib" + File.separator + "asterix-app-"
+                + asterixInstance.getAsterixVersion() + ".jar");
+        File replacementFile = new File(asterixInstanceDir + File.separator + "log.properties");
+        replaceInJar(sourceJar2, txnLogPropertyFile, replacementFile);
+
+        new File(asterixInstanceDir + File.separator + "log.properties").delete();
+        String asterixZipName = InstallerDriver.getAsterixZip().substring(
+                InstallerDriver.getAsterixZip().lastIndexOf(File.separator) + 1);
+        zipDir(new File(asterixInstanceDir), new File(asterixInstanceDir + File.separator + asterixZipName));
+        return asterixInstanceDir + File.separator + asterixZipName;
     }
 
     public static void addLibraryToAsterixZip(AsterixInstance asterixInstance, String dataverseName,
@@ -111,7 +151,6 @@
 
     public static String getNodeDirectories(String asterixInstanceName, Node node, Cluster cluster) {
         String storeDataSubDir = asterixInstanceName + File.separator + "data" + File.separator;
-        String storeLibrarySubDir = asterixInstanceName + File.separator + "library" + File.separator;
         String[] storeDirs = null;
         StringBuffer nodeDataStore = new StringBuffer();
         String storeDirValue = node.getStore();
@@ -121,7 +160,6 @@
                 throw new IllegalStateException(" Store not defined for node " + node.getId());
             }
             storeDataSubDir = node.getId() + File.separator + storeDataSubDir;
-            storeLibrarySubDir = node.getId() + File.separator + storeLibrarySubDir;
         }
 
         storeDirs = storeDirValue.split(",");
@@ -143,19 +181,40 @@
         conf.append("MetadataNode=" + asterixInstanceName + "_" + metadataNodeId + "\n");
         conf.append("NewUniverse=" + newData + "\n");
 
+        String storeDir = null;
         for (Node node : cluster.getNode()) {
-            String nodeDir = getNodeDirectories(asterixInstance.getName(), node, cluster);
-            conf.append(asterixInstanceName + "_" + node.getId() + ".stores" + "=" + nodeDir + "\n");
+            storeDir = node.getStore() == null ? cluster.getStore() : node.getStore();
+            conf.append(asterixInstanceName + "_" + node.getId() + ".stores" + "=" + storeDir + "\n");
         }
+
         Properties asterixConfProp = asterixInstance.getConfiguration();
         String outputDir = asterixConfProp.getProperty("output_dir");
         conf.append("OutputDir=" + outputDir);
+
         File asterixConfDir = new File(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName);
         asterixConfDir.mkdirs();
         dumpToFile(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName + File.separator
                 + "test.properties", conf.toString());
     }
 
+    private static void writeAsterixLogConfigurationFile(String asterixInstanceName, Cluster cluster,
+            Properties logProperties) throws IOException {
+        StringBuffer conf = new StringBuffer();
+        for (Map.Entry<Object, Object> p : logProperties.entrySet()) {
+            conf.append(p.getKey() + "=" + p.getValue() + "\n");
+        }
+
+        for (Node node : cluster.getNode()) {
+            String iodevices = node.getIodevices() == null ? cluster.getIodevices() : node.getIodevices();
+            String store = node.getStore() == null ? cluster.getStore() : node.getStore();
+            String txnLogDir = iodevices.split(",")[0].trim() + File.separator + store;
+            conf.append(asterixInstanceName + "_" + node.getId() + "." + "txnLogDir=" + txnLogDir + "\n");
+        }
+        dumpToFile(InstallerDriver.getAsterixDir() + File.separator + asterixInstanceName + File.separator
+                + "log.properties", conf.toString());
+
+    }
+
     public static Properties getAsterixConfiguration(String asterixConf) throws FileNotFoundException, IOException {
         Properties prop = new Properties();
         prop.load(new FileInputStream(asterixConf));
@@ -301,21 +360,21 @@
     public static void evaluateConflictWithOtherInstances(AsterixInstance instance) throws Exception {
         List<AsterixInstance> existingInstances = ServiceProvider.INSTANCE.getLookupService().getAsterixInstances();
         List<String> usedIps = new ArrayList<String>();
-        String masterIp = instance.getCluster().getMasterNode().getIp();
+        String masterIp = instance.getCluster().getMasterNode().getClusterIp();
         for (Node node : instance.getCluster().getNode()) {
-            usedIps.add(node.getIp());
+            usedIps.add(node.getClusterIp());
         }
-        usedIps.add(instance.getCluster().getMasterNode().getIp());
+        usedIps.add(instance.getCluster().getMasterNode().getClusterIp());
         boolean conflictFound = false;
         AsterixInstance conflictingInstance = null;
         for (AsterixInstance existing : existingInstances) {
-            conflictFound = existing.getCluster().getMasterNode().getIp().equals(masterIp);
+            conflictFound = existing.getCluster().getMasterNode().getClusterIp().equals(masterIp);
             if (conflictFound) {
                 conflictingInstance = existing;
                 break;
             }
             for (Node n : existing.getCluster().getNode()) {
-                if (usedIps.contains(n.getIp())) {
+                if (usedIps.contains(n.getClusterIp())) {
                     conflictFound = true;
                     conflictingInstance = existing;
                     break;
@@ -341,7 +400,7 @@
         }
         ProcessBuilder pb = new ProcessBuilder(pargs);
         pb.environment().putAll(EventDriver.getEnvironment());
-        pb.environment().put("IP_LOCATION", EventDriver.CLIENT_NODE.getIp());
+        pb.environment().put("IP_LOCATION", EventDriver.CLIENT_NODE.getClusterIp());
         Process p = pb.start();
         BufferedInputStream bis = new BufferedInputStream(p.getInputStream());
         StringWriter writer = new StringWriter();
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
index 7c7a792..05a92f2 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/error/VerificationUtil.java
@@ -29,93 +29,110 @@
 
 public class VerificationUtil {
 
-    private static final String VERIFY_SCRIPT_PATH = InstallerDriver.getManagixHome() + File.separator
-            + InstallerDriver.MANAGIX_INTERNAL_DIR + File.separator + "scripts" + File.separator + "verify.sh";
+	private static final String VERIFY_SCRIPT_PATH = InstallerDriver
+			.getManagixHome()
+			+ File.separator
+			+ InstallerDriver.MANAGIX_INTERNAL_DIR
+			+ File.separator
+			+ "scripts"
+			+ File.separator + "verify.sh";
 
-    public static AsterixRuntimeState getAsterixRuntimeState(AsterixInstance instance) throws Exception {
+	public static AsterixRuntimeState getAsterixRuntimeState(
+			AsterixInstance instance) throws Exception {
 
-        Cluster cluster = instance.getCluster();
-        List<String> args = new ArrayList<String>();
-        args.add(instance.getName());
-        args.add(instance.getCluster().getMasterNode().getIp());
-        for (Node node : cluster.getNode()) {
-            args.add(node.getIp());
-            args.add(instance.getName() + "_" + node.getId());
-        }
+		Cluster cluster = instance.getCluster();
+		List<String> args = new ArrayList<String>();
+		args.add(instance.getName());
+		args.add(instance.getCluster().getMasterNode().getClusterIp());
+		for (Node node : cluster.getNode()) {
+			args.add(node.getClusterIp());
+			args.add(instance.getName() + "_" + node.getId());
+		}
 
-        String output = InstallerUtil.executeLocalScript(VERIFY_SCRIPT_PATH, args);
-        boolean ccRunning = true;
-        List<String> failedNCs = new ArrayList<String>();
-        String[] infoFields;
-        ProcessInfo pInfo;
-        List<ProcessInfo> processes = new ArrayList<ProcessInfo>();
+		String output = InstallerUtil.executeLocalScript(VERIFY_SCRIPT_PATH,
+				args);
+		boolean ccRunning = true;
+		List<String> failedNCs = new ArrayList<String>();
+		String[] infoFields;
+		ProcessInfo pInfo;
+		List<ProcessInfo> processes = new ArrayList<ProcessInfo>();
 
-        for (String line : output.split("\n")) {
-            infoFields = line.split(":");
-            try {
-                int pid = Integer.parseInt(infoFields[3]);
-                pInfo = new ProcessInfo(infoFields[0], infoFields[1], pid);
-                processes.add(pInfo);
-            } catch (Exception e) {
-                if (infoFields[0].equalsIgnoreCase("CC")) {
-                    ccRunning = false;
-                } else {
-                    failedNCs.add(infoFields[1]);
-                }
-            }
-        }
-        return new AsterixRuntimeState(processes, failedNCs, ccRunning);
-    }
+		for (String line : output.split("\n")) {
+			String nodeid = null;
+			infoFields = line.split(":");
+			try {
+				int pid = Integer.parseInt(infoFields[3]);
+				if (infoFields[0].equals("NC")) {
+					nodeid = infoFields[2].split("_")[1];
+				} else {
+					nodeid = instance.getCluster().getMasterNode().getId();
+				}
+				pInfo = new ProcessInfo(infoFields[0], infoFields[1], nodeid,
+						pid);
+				processes.add(pInfo);
+			} catch (Exception e) {
+				if (infoFields[0].equalsIgnoreCase("CC")) {
+					ccRunning = false;
+				} else {
+					failedNCs.add(infoFields[1]);
+				}
+			}
+		}
+		return new AsterixRuntimeState(processes, failedNCs, ccRunning);
+	}
 
-    public static void updateInstanceWithRuntimeDescription(AsterixInstance instance, AsterixRuntimeState state,
-            boolean expectedRunning) {
-        StringBuffer summary = new StringBuffer();
-        if (expectedRunning) {
-            if (!state.isCcRunning()) {
-                summary.append("Cluster Controller not running at " + instance.getCluster().getMasterNode().getIp()
-                        + "\n");
-                instance.setState(State.UNUSABLE);
-            }
-            if (state.getFailedNCs() != null && !state.getFailedNCs().isEmpty()) {
-                summary.append("Node Controller not running at the following nodes" + "\n");
-                for (String failedNC : state.getFailedNCs()) {
-                    summary.append(failedNC + "\n");
-                }
-                instance.setState(State.UNUSABLE);
-            }
-            if (!(instance.getState().equals(State.UNUSABLE))) {
-                instance.setState(State.ACTIVE);
-            }
-        } else {
-            if (state.getProcesses() != null && state.getProcesses().size() > 0) {
-                summary.append("Following process still running " + "\n");
-                for (ProcessInfo pInfo : state.getProcesses()) {
-                    summary.append(pInfo + "\n");
-                }
-                instance.setState(State.UNUSABLE);
-            } else {
-                instance.setState(State.INACTIVE);
-            }
-        }
-        state.setSummary(summary.toString());
-        instance.setAsterixRuntimeStates(state);
-    }
+	public static void updateInstanceWithRuntimeDescription(
+			AsterixInstance instance, AsterixRuntimeState state,
+			boolean expectedRunning) {
+		StringBuffer summary = new StringBuffer();
+		if (expectedRunning) {
+			if (!state.isCcRunning()) {
+				summary.append("Cluster Controller not running at "
+						+ instance.getCluster().getMasterNode().getId() + "\n");
+				instance.setState(State.UNUSABLE);
+			}
+			if (state.getFailedNCs() != null && !state.getFailedNCs().isEmpty()) {
+				summary.append("Node Controller not running at the following nodes"
+						+ "\n");
+				for (String failedNC : state.getFailedNCs()) {
+					summary.append(failedNC + "\n");
+				}
+				instance.setState(State.UNUSABLE);
+			}
+			if (!(instance.getState().equals(State.UNUSABLE))) {
+				instance.setState(State.ACTIVE);
+			}
+		} else {
+			if (state.getProcesses() != null && state.getProcesses().size() > 0) {
+				summary.append("Following process still running " + "\n");
+				for (ProcessInfo pInfo : state.getProcesses()) {
+					summary.append(pInfo + "\n");
+				}
+				instance.setState(State.UNUSABLE);
+			} else {
+				instance.setState(State.INACTIVE);
+			}
+		}
+		state.setSummary(summary.toString());
+		instance.setAsterixRuntimeStates(state);
+	}
 
-    public static void verifyBackupRestoreConfiguration(String hdfsUrl, String hadoopVersion, String hdfsBackupDir)
-            throws Exception {
-        StringBuffer errorCheck = new StringBuffer();
-        if (hdfsUrl == null || hdfsUrl.length() == 0) {
-            errorCheck.append("\n HDFS Url not configured");
-        }
-        if (hadoopVersion == null || hadoopVersion.length() == 0) {
-            errorCheck.append("\n HDFS version not configured");
-        }
-        if (hdfsBackupDir == null || hdfsBackupDir.length() == 0) {
-            errorCheck.append("\n HDFS backup directory not configured");
-        }
-        if (errorCheck.length() > 0) {
-            throw new Exception("Incomplete hdfs configuration in " + InstallerDriver.getManagixHome() + File.separator
-                    + InstallerDriver.MANAGIX_CONF_XML + errorCheck);
-        }
-    }
+	public static void verifyBackupRestoreConfiguration(String hdfsUrl,
+			String hadoopVersion, String hdfsBackupDir) throws Exception {
+		StringBuffer errorCheck = new StringBuffer();
+		if (hdfsUrl == null || hdfsUrl.length() == 0) {
+			errorCheck.append("\n HDFS Url not configured");
+		}
+		if (hadoopVersion == null || hadoopVersion.length() == 0) {
+			errorCheck.append("\n HDFS version not configured");
+		}
+		if (hdfsBackupDir == null || hdfsBackupDir.length() == 0) {
+			errorCheck.append("\n HDFS backup directory not configured");
+		}
+		if (errorCheck.length() > 0) {
+			throw new Exception("Incomplete hdfs configuration in "
+					+ InstallerDriver.getManagixHome() + File.separator
+					+ InstallerDriver.MANAGIX_CONF_XML + errorCheck);
+		}
+	}
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
index fc53304..2184a63 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/events/PatternCreator.java
@@ -29,6 +29,7 @@
 import edu.uci.ics.asterix.event.schema.pattern.Pattern;
 import edu.uci.ics.asterix.event.schema.pattern.Patterns;
 import edu.uci.ics.asterix.event.schema.pattern.Value;
+import edu.uci.ics.asterix.installer.command.BackupCommand;
 import edu.uci.ics.asterix.installer.command.StopCommand;
 import edu.uci.ics.asterix.installer.driver.InstallerDriver;
 import edu.uci.ics.asterix.installer.error.VerificationUtil;
@@ -52,7 +53,7 @@
 	public Patterns getStartAsterixPattern(String asterixInstanceName,
 			Cluster cluster) throws Exception {
 		String ccLocationId = cluster.getMasterNode().getId();
-		String ccLocationIp = cluster.getMasterNode().getIp();
+		String ccLocationIp = cluster.getMasterNode().getClusterIp();
 
 		String destDir = cluster.getWorkingDir().getDir() + File.separator
 				+ "asterix";
@@ -70,12 +71,15 @@
 		for (Node node : cluster.getNode()) {
 			if (copyHyracksToNC) {
 				Pattern copyHyracksForNC = createCopyHyracksPattern(
-						asterixInstanceName, cluster, node.getIp(), destDir);
+						asterixInstanceName, cluster, node.getClusterIp(),
+						destDir);
 				ps.add(copyHyracksForNC);
 			}
+			String iodevices = node.getIodevices() == null ? cluster
+					.getIodevices() : node.getIodevices();
 			Pattern createNC = createNCStartPattern(cluster.getMasterNode()
-					.getIp(), node.getId(),
-					asterixInstanceName + "_" + node.getId());
+					.getClusterIp(), node.getId(), asterixInstanceName + "_"
+					+ node.getId(), iodevices);
 			addInitialDelay(createNC, 4, "sec");
 			ps.add(createNC);
 		}
@@ -144,7 +148,6 @@
 	private Patterns getHDFSBackUpAsterixPattern(AsterixInstance instance,
 			Backup backupConf) throws Exception {
 		Cluster cluster = instance.getCluster();
-		String clusterStore = instance.getCluster().getStore();
 		String hdfsUrl = backupConf.getHdfs().getUrl();
 		String hadoopVersion = backupConf.getHdfs().getVersion();
 		String hdfsBackupDir = backupConf.getBackupDir();
@@ -152,16 +155,21 @@
 				hadoopVersion, hdfsBackupDir);
 		String workingDir = cluster.getWorkingDir().getDir();
 		String backupId = "" + instance.getBackupInfo().size();
-		String nodeStore;
+		String store;
 		String pargs;
+		String iodevices;
 		List<Pattern> patternList = new ArrayList<Pattern>();
 		for (Node node : cluster.getNode()) {
 			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			nodeStore = node.getStore() == null ? clusterStore : node
+			iodevices = node.getIodevices() == null ? instance.getCluster()
+					.getIodevices() : node.getIodevices();
+			store = node.getStore() == null ? cluster.getStore() : node
 					.getStore();
-			pargs = workingDir + " " + instance.getName() + " " + nodeStore
-					+ " " + backupId + " " + hdfsBackupDir + " " + "hdfs" + " "
-					+ node.getId() + " " + hdfsUrl + " " + hadoopVersion;
+			pargs = workingDir + " " + instance.getName() + " " + iodevices
+					+ " " + store + " "
+					+ BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + backupId
+					+ " " + hdfsBackupDir + " " + "hdfs" + " " + node.getId()
+					+ " " + hdfsUrl + " " + hadoopVersion;
 			Event event = new Event("backup", nodeid, pargs);
 			patternList.add(new Pattern(null, 1, null, event));
 		}
@@ -171,20 +179,23 @@
 	private Patterns getLocalBackUpAsterixPattern(AsterixInstance instance,
 			Backup backupConf) throws Exception {
 		Cluster cluster = instance.getCluster();
-		String clusterStore = instance.getCluster().getStore();
 		String backupDir = backupConf.getBackupDir();
 		String workingDir = cluster.getWorkingDir().getDir();
 		String backupId = "" + instance.getBackupInfo().size();
-		String nodeStore;
+		String iodevices;
+		String store;
 		String pargs;
 		List<Pattern> patternList = new ArrayList<Pattern>();
 		for (Node node : cluster.getNode()) {
 			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			nodeStore = node.getStore() == null ? clusterStore : node
+			iodevices = node.getIodevices() == null ? instance.getCluster()
+					.getIodevices() : node.getIodevices();
+			store = node.getStore() == null ? cluster.getStore() : node
 					.getStore();
-			pargs = workingDir + " " + instance.getName() + " " + nodeStore
-					+ " " + backupId + " " + backupDir + " " + "local" + " "
-					+ node.getId();
+			pargs = workingDir + " " + instance.getName() + " " + iodevices
+					+ " " + store + " "
+					+ BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + backupId
+					+ " " + backupDir + " " + "local" + " " + node.getId();
 			Event event = new Event("backup", nodeid, pargs);
 			patternList.add(new Pattern(null, 1, null, event));
 		}
@@ -208,10 +219,14 @@
 		List<Pattern> patternList = new ArrayList<Pattern>();
 		for (Node node : cluster.getNode()) {
 			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
+			String iodevices = node.getIodevices() == null ? cluster
+					.getIodevices() : node.getIodevices();
 			nodeStore = node.getStore() == null ? clusterStore : node
 					.getStore();
-			pargs = workingDir + " " + instance.getName() + " " + nodeStore
-					+ " " + backupId + " " + hdfsBackupDir + " " + "hdfs" + " "
+			pargs = workingDir + " " + instance.getName() + " " + iodevices
+					+ " " + nodeStore + " "
+					+ BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + backupId
+					+ " " + " " + hdfsBackupDir + " " + "hdfs" + " "
 					+ node.getId() + " " + hdfsUrl + " " + hadoopVersion;
 			Event event = new Event("restore", nodeid, pargs);
 			patternList.add(new Pattern(null, 1, null, event));
@@ -231,11 +246,14 @@
 		List<Pattern> patternList = new ArrayList<Pattern>();
 		for (Node node : cluster.getNode()) {
 			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
+			String iodevices = node.getIodevices() == null ? cluster
+					.getIodevices() : node.getIodevices();
 			nodeStore = node.getStore() == null ? clusterStore : node
 					.getStore();
-			pargs = workingDir + " " + instance.getName() + " " + nodeStore
-					+ " " + backupId + " " + backupDir + " " + "local" + " "
-					+ node.getId();
+			pargs = workingDir + " " + instance.getName() + " " + iodevices
+					+ " " + nodeStore + " "
+					+ BackupCommand.ASTERIX_ROOT_METADATA_DIR + " " + backupId
+					+ " " + backupDir + " " + "local" + " " + node.getId();
 			Event event = new Event("restore", nodeid, pargs);
 			patternList.add(new Pattern(null, 1, null, event));
 		}
@@ -261,7 +279,7 @@
 		String username = cluster.getUsername() != null ? cluster.getUsername()
 				: System.getProperty("user.name");
 		String pargs = username + " " + hadoopDir.getAbsolutePath() + " "
-				+ cluster.getMasterNode().getIp() + " " + workingDir;
+				+ cluster.getMasterNode().getClusterIp() + " " + workingDir;
 		Event event = new Event("directory_transfer", nodeid, pargs);
 		Pattern p = new Pattern(null, 1, null, event);
 		addInitialDelay(p, 2, "sec");
@@ -272,8 +290,8 @@
 			for (Node node : cluster.getNode()) {
 				nodeid = new Nodeid(new Value(null, node.getId()));
 				pargs = cluster.getUsername() + " "
-						+ hadoopDir.getAbsolutePath() + " " + node.getIp()
-						+ " " + workingDir;
+						+ hadoopDir.getAbsolutePath() + " "
+						+ node.getClusterIp() + " " + workingDir;
 				event = new Event("directory_transfer", nodeid, pargs);
 				p = new Pattern(null, 1, null, event);
 				addInitialDelay(p, 2, "sec");
@@ -322,6 +340,8 @@
 		}
 		patternList.addAll(createRemoveAsterixLogDirPattern(instance)
 				.getPattern());
+		patternList.addAll(createRemoveAsterixRootMetadata(instance)
+				.getPattern());
 		Patterns patterns = new Patterns(patternList);
 		return patterns;
 	}
@@ -357,13 +377,13 @@
 		String pargs = pathToDelete;
 		List<String> removedBackupDirs = new ArrayList<String>();
 		for (Node node : cluster.getNode()) {
-			if (removedBackupDirs.contains(node.getIp())) {
+			if (removedBackupDirs.contains(node.getClusterIp())) {
 				continue;
 			}
 			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
 			Event event = new Event("file_delete", nodeid, pargs);
 			patternList.add(new Pattern(null, 1, null, event));
-			removedBackupDirs.add(node.getIp());
+			removedBackupDirs.add(node.getClusterIp());
 		}
 
 		Patterns patterns = new Patterns(patternList);
@@ -392,6 +412,28 @@
 		return patterns;
 	}
 
+	private Patterns createRemoveAsterixRootMetadata(AsterixInstance instance)
+			throws Exception {
+		List<Pattern> patternList = new ArrayList<Pattern>();
+		Cluster cluster = instance.getCluster();
+		Nodeid nodeid = null;
+		String pargs = null;
+		Event event = null;
+		for (Node node : cluster.getNode()) {
+			String iodevices = node.getIodevices() == null ? cluster
+					.getIodevices() : node.getIodevices();
+			String primaryIODevice = iodevices.split(",")[0].trim();
+			pargs = primaryIODevice + File.separator
+					+ BackupCommand.ASTERIX_ROOT_METADATA_DIR;
+			nodeid = new Nodeid(new Value(null, node.getId()));
+			event = new Event("file_delete", nodeid, pargs);
+			patternList.add(new Pattern(null, 1, null, event));
+		}
+
+		Patterns patterns = new Patterns(patternList);
+		return patterns;
+	}
+
 	private Patterns createRemoveAsterixLogDirPattern(AsterixInstance instance)
 			throws Exception {
 		List<Pattern> patternList = new ArrayList<Pattern>();
@@ -402,13 +444,12 @@
 		Event event = new Event("file_delete", nodeid, pargs);
 		patternList.add(new Pattern(null, 1, null, event));
 
-		if (!cluster.getWorkingDir().isNFS()) {
-			for (Node node : cluster.getNode()) {
-				nodeid = new Nodeid(new Value(null, node.getId()));
-				event = new Event("file_delete", nodeid, pargs);
-				patternList.add(new Pattern(null, 1, null, event));
-			}
+		for (Node node : cluster.getNode()) {
+			nodeid = new Nodeid(new Value(null, node.getId()));
+			event = new Event("file_delete", nodeid, pargs);
+			patternList.add(new Pattern(null, 1, null, event));
 		}
+
 		Patterns patterns = new Patterns(patternList);
 		return patterns;
 	}
@@ -421,21 +462,17 @@
 
 		for (Node node : cluster.getNode()) {
 			Nodeid nodeid = new Nodeid(new Value(null, node.getId()));
-			String[] nodeStores;
-			if (node.getStore() != null) {
-				nodeStores = node.getStore().trim().split(",");
-				for (String ns : nodeStores) {
-					pargs = ns + File.separator + instance.getName();
-				}
-			} else {
-				nodeStores = cluster.getStore().trim().split(",");
-				for (String ns : nodeStores) {
-					pargs = ns + File.separator + node.getId() + File.separator
-							+ instance.getName();
-				}
+			String[] nodeIODevices;
+			String iodevices = node.getIodevices() == null ? cluster
+					.getIodevices() : node.getIodevices();
+			nodeIODevices = iodevices.trim().split(",");
+			for (String nodeIODevice : nodeIODevices) {
+				String nodeStore = node.getStore() == null ? cluster.getStore()
+						: node.getStore();
+				pargs = nodeIODevice.trim() + File.separator + nodeStore;
+				Event event = new Event("file_delete", nodeid, pargs);
+				patternList.add(new Pattern(null, 1, null, event));
 			}
-			Event event = new Event("file_delete", nodeid, pargs);
-			patternList.add(new Pattern(null, 1, null, event));
 		}
 		Patterns patterns = new Patterns(patternList);
 		return patterns;
@@ -473,10 +510,10 @@
 	}
 
 	public Pattern createNCStartPattern(String ccHost, String hostId,
-			String nodeControllerId) {
+			String nodeControllerId, String iodevices) {
 		Nodeid nodeid = new Nodeid(new Value(null, hostId));
-		Event event = new Event("node_join", nodeid, ccHost + " "
-				+ nodeControllerId);
+		String pargs = ccHost + " " + nodeControllerId + " " + iodevices;
+		Event event = new Event("node_join", nodeid, pargs);
 		return new Pattern(null, 1, null, event);
 	}
 
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
index d3ba758..9f8b28d 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/AsterixInstance.java
@@ -25,157 +25,159 @@
 
 public class AsterixInstance implements Serializable {
 
-    private static final long serialVersionUID = 2874439550187520449L;
+	private static final long serialVersionUID = 2874439550187520449L;
 
-    public enum State {
-        ACTIVE,
-        INACTIVE,
-        UNUSABLE
-    }
+	public enum State {
+		ACTIVE, INACTIVE, UNUSABLE
+	}
 
-    private final Cluster cluster;
-    private final String name;
-    private final Date createdTimestamp;
-    private Date stateChangeTimestamp;
-    private Date modifiedTimestamp;
-    private Properties configuration;
-    private State state;
-    private final String metadataNodeId;
-    private final String asterixVersion;
-    private final List<BackupInfo> backupInfo;
-    private final String webInterfaceUrl;
-    private AsterixRuntimeState runtimeState;
-    private State previousState;
+	private final Cluster cluster;
+	private final String name;
+	private final Date createdTimestamp;
+	private Date stateChangeTimestamp;
+	private Date modifiedTimestamp;
+	private Properties configuration;
+	private State state;
+	private final String metadataNodeId;
+	private final String asterixVersion;
+	private final List<BackupInfo> backupInfo;
+	private final String webInterfaceUrl;
+	private AsterixRuntimeState runtimeState;
+	private State previousState;
 
-    public AsterixInstance(String name, Cluster cluster, Properties configuration, String metadataNodeId,
-            String asterixVersion) {
-        this.name = name;
-        this.cluster = cluster;
-        this.configuration = configuration;
-        this.metadataNodeId = metadataNodeId;
-        this.state = State.ACTIVE;
-        this.previousState = State.UNUSABLE;
-        this.asterixVersion = asterixVersion;
-        this.createdTimestamp = new Date();
-        this.backupInfo = new ArrayList<BackupInfo>();
-        this.webInterfaceUrl = "http://" + cluster.getMasterNode().getIp() + ":" + 19001;
-    }
+	public AsterixInstance(String name, Cluster cluster,
+			Properties configuration, String metadataNodeId,
+			String asterixVersion) {
+		this.name = name;
+		this.cluster = cluster;
+		this.configuration = configuration;
+		this.metadataNodeId = metadataNodeId;
+		this.state = State.ACTIVE;
+		this.previousState = State.UNUSABLE;
+		this.asterixVersion = asterixVersion;
+		this.createdTimestamp = new Date();
+		this.backupInfo = new ArrayList<BackupInfo>();
+		this.webInterfaceUrl = "http://"
+				+ cluster.getMasterNode().getClusterIp() + ":" + 19001;
+	}
 
-    public Date getModifiedTimestamp() {
-        return stateChangeTimestamp;
-    }
+	public Date getModifiedTimestamp() {
+		return stateChangeTimestamp;
+	}
 
-    public Properties getConfiguration() {
-        return configuration;
-    }
+	public Properties getConfiguration() {
+		return configuration;
+	}
 
-    public void setConfiguration(Properties properties) {
-        this.configuration = properties;
-    }
+	public void setConfiguration(Properties properties) {
+		this.configuration = properties;
+	}
 
-    public State getState() {
-        return state;
-    }
+	public State getState() {
+		return state;
+	}
 
-    public void setState(State state) {
-        this.previousState = this.state;
-        this.state = state;
-    }
+	public void setState(State state) {
+		this.previousState = this.state;
+		this.state = state;
+	}
 
-    public Cluster getCluster() {
-        return cluster;
-    }
+	public Cluster getCluster() {
+		return cluster;
+	}
 
-    public String getName() {
-        return name;
-    }
+	public String getName() {
+		return name;
+	}
 
-    public Date getCreatedTimestamp() {
-        return createdTimestamp;
-    }
+	public Date getCreatedTimestamp() {
+		return createdTimestamp;
+	}
 
-    public Date getStateChangeTimestamp() {
-        return stateChangeTimestamp;
-    }
+	public Date getStateChangeTimestamp() {
+		return stateChangeTimestamp;
+	}
 
-    public void setStateChangeTimestamp(Date stateChangeTimestamp) {
-        this.stateChangeTimestamp = stateChangeTimestamp;
-    }
+	public void setStateChangeTimestamp(Date stateChangeTimestamp) {
+		this.stateChangeTimestamp = stateChangeTimestamp;
+	}
 
-    public void setModifiedTimestamp(Date modifiedTimestamp) {
-        this.modifiedTimestamp = modifiedTimestamp;
-    }
+	public void setModifiedTimestamp(Date modifiedTimestamp) {
+		this.modifiedTimestamp = modifiedTimestamp;
+	}
 
-    public String getMetadataNodeId() {
-        return metadataNodeId;
-    }
+	public String getMetadataNodeId() {
+		return metadataNodeId;
+	}
 
-    public String getAsterixVersion() {
-        return asterixVersion;
-    }
+	public String getAsterixVersion() {
+		return asterixVersion;
+	}
 
-    public String getDescription(boolean detailed) {
-        StringBuffer buffer = new StringBuffer();
-        buffer.append("Name:" + name + "\n");
-        buffer.append("Created:" + createdTimestamp + "\n");
-        buffer.append("Web-Url:" + webInterfaceUrl + "\n");
-        buffer.append("State:" + state);
-        if (!state.equals(State.UNUSABLE) && stateChangeTimestamp != null) {
-            buffer.append(" (" + stateChangeTimestamp + ")" + "\n");
-        } else {
-            buffer.append("\n");
-        }
-        if (modifiedTimestamp != null) {
-            buffer.append("Last modified timestamp:" + modifiedTimestamp + "\n");
-        }
+	public String getDescription(boolean detailed) {
+		StringBuffer buffer = new StringBuffer();
+		buffer.append("Name:" + name + "\n");
+		buffer.append("Created:" + createdTimestamp + "\n");
+		buffer.append("Web-Url:" + webInterfaceUrl + "\n");
+		buffer.append("State:" + state);
+		if (!state.equals(State.UNUSABLE) && stateChangeTimestamp != null) {
+			buffer.append(" (" + stateChangeTimestamp + ")" + "\n");
+		} else {
+			buffer.append("\n");
+		}
+		if (modifiedTimestamp != null) {
+			buffer.append("Last modified timestamp:" + modifiedTimestamp + "\n");
+		}
 
-        if (runtimeState.getSummary() != null && runtimeState.getSummary().length() > 0) {
-            buffer.append("\nWARNING!:" + runtimeState.getSummary() + "\n");
-        }
-        if (detailed) {
-            addDetailedInformation(buffer);
-        }
-        return buffer.toString();
-    }
+		if (runtimeState.getSummary() != null
+				&& runtimeState.getSummary().length() > 0) {
+			buffer.append("\nWARNING!:" + runtimeState.getSummary() + "\n");
+		}
+		if (detailed) {
+			addDetailedInformation(buffer);
+		}
+		return buffer.toString();
+	}
 
-    public List<BackupInfo> getBackupInfo() {
-        return backupInfo;
-    }
+	public List<BackupInfo> getBackupInfo() {
+		return backupInfo;
+	}
 
-    public String getWebInterfaceUrl() {
-        return webInterfaceUrl;
-    }
+	public String getWebInterfaceUrl() {
+		return webInterfaceUrl;
+	}
 
-    public AsterixRuntimeState getAsterixRuntimeState() {
-        return runtimeState;
-    }
+	public AsterixRuntimeState getAsterixRuntimeState() {
+		return runtimeState;
+	}
 
-    public void setAsterixRuntimeStates(AsterixRuntimeState runtimeState) {
-        this.runtimeState = runtimeState;
-    }
+	public void setAsterixRuntimeStates(AsterixRuntimeState runtimeState) {
+		this.runtimeState = runtimeState;
+	}
 
-    private void addDetailedInformation(StringBuffer buffer) {
-        buffer.append("Master node:" + cluster.getMasterNode().getId() + ":" + cluster.getMasterNode().getIp() + "\n");
-        for (Node node : cluster.getNode()) {
-            buffer.append(node.getId() + ":" + node.getIp() + "\n");
-        }
+	private void addDetailedInformation(StringBuffer buffer) {
+		buffer.append("Master node:" + cluster.getMasterNode().getId() + ":"
+				+ cluster.getMasterNode().getClusterIp() + "\n");
+		for (Node node : cluster.getNode()) {
+			buffer.append(node.getId() + ":" + node.getClusterIp() + "\n");
+		}
 
-        if (backupInfo != null && backupInfo.size() > 0) {
-            for (BackupInfo info : backupInfo) {
-                buffer.append(info + "\n");
-            }
-        }
-        buffer.append("\n");
-        buffer.append("Asterix version:" + asterixVersion + "\n");
-        buffer.append("Metadata Node:" + metadataNodeId + "\n");
-        buffer.append("Processes" + "\n");
-        for (ProcessInfo pInfo : runtimeState.getProcesses()) {
-            buffer.append(pInfo + "\n");
-        }
+		if (backupInfo != null && backupInfo.size() > 0) {
+			for (BackupInfo info : backupInfo) {
+				buffer.append(info + "\n");
+			}
+		}
+		buffer.append("\n");
+		buffer.append("Asterix version:" + asterixVersion + "\n");
+		buffer.append("Metadata Node:" + metadataNodeId + "\n");
+		buffer.append("Processes" + "\n");
+		for (ProcessInfo pInfo : runtimeState.getProcesses()) {
+			buffer.append(pInfo + "\n");
+		}
 
-    }
+	}
 
-    public State getPreviousState() {
-        return previousState;
-    }
+	public State getPreviousState() {
+		return previousState;
+	}
 }
diff --git a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
index 2880671..42ee112 100644
--- a/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
+++ b/asterix-installer/src/main/java/edu/uci/ics/asterix/installer/model/ProcessInfo.java
@@ -21,11 +21,13 @@
     private static final long serialVersionUID = 304186774065853730L;
     private final String processName;
     private final String host;
+    private final String nodeId;
     private final int processId;
 
-    public ProcessInfo(String processName, String host, int processId) {
+    public ProcessInfo(String processName, String host, String nodeId, int processId) {
         this.processName = processName;
         this.host = host;
+        this.nodeId = nodeId;
         this.processId = processId;
     }
 
@@ -41,8 +43,12 @@
         return processId;
     }
 
+    public String getNodeId() {
+        return nodeId;
+    }
+
     public String toString() {
-        return processName + " at " + host + " [ " + processId + " ] ";
+        return processName + " at " + nodeId + " [ " + processId + " ] ";
     }
 
 }
diff --git a/asterix-installer/src/main/resources/clusters/local/local.xml b/asterix-installer/src/main/resources/clusters/local/local.xml
index 7a8e78f..b6589d0 100644
--- a/asterix-installer/src/main/resources/clusters/local/local.xml
+++ b/asterix-installer/src/main/resources/clusters/local/local.xml
@@ -1,20 +1,21 @@
 <cluster xmlns="cluster">
   <name>local</name>
   <workingDir>
-     <dir>/tmp/asterix-installer</dir>
-     <NFS>true</NFS> 
+    <dir>/tmp/asterix-installer</dir>
+    <NFS>true</NFS>
   </workingDir>
   <logdir>/tmp/asterix/logs</logdir>
-  <store>/tmp/asterix/storage</store>
+  <iodevices>/tmp</iodevices>
+  <store>asterix/storage</store>
   <java_home></java_home>
   <java_heap>1024m</java_heap>
   <master-node>
-     <id>master</id>
-     <ip>127.0.0.1</ip>
-     <cluster-ip>127.0.0.1</cluster-ip>
+    <id>master</id>
+    <client-ip>127.0.0.1</client-ip>
+    <cluster-ip>127.0.0.1</cluster-ip>
   </master-node>
   <node>
-     <id>node1</id>
-     <ip>127.0.0.1</ip>
+    <id>node1</id>
+    <cluster-ip>127.0.0.1</cluster-ip>
   </node>
 </cluster>
diff --git a/asterix-installer/src/main/resources/conf/managix-conf.xml b/asterix-installer/src/main/resources/conf/managix-conf.xml
index ed3f8a2..10a62a9 100644
--- a/asterix-installer/src/main/resources/conf/managix-conf.xml
+++ b/asterix-installer/src/main/resources/conf/managix-conf.xml
@@ -10,7 +10,7 @@
     <homeDir>/tmp/zookeeper</homeDir>
     <clientPort>2900</clientPort>
     <servers>
-      <server>localhost</server>
+      <server>127.0.0.1</server>
     </servers>
   </zookeeper>
 </configuration>