Merged kisskys/master_serialized_log_appender into kisskys_salsubaiee/storage_log_manager_fixes.
diff --git a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
index f716387..a9b3881 100644
--- a/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
+++ b/asterix-app/src/main/java/edu/uci/ics/asterix/file/SecondaryIndexCreator.java
@@ -26,7 +26,7 @@
 import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.context.TransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryBooleanInspectorImpl;
 import edu.uci.ics.asterix.formats.nontagged.AqlBinaryComparatorFactoryProvider;
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
index fcc6dad..da08cd8 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/AbstractLSMIOOperationCallback.java
@@ -45,7 +45,7 @@
         opTracker.resetLSNs();
     }
 
-    protected abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
+    public abstract long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException;
 
     protected void putLSNIntoMetadata(ITreeIndex treeIndex, List<ILSMComponent> oldComponents)
             throws HyracksDataException {
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
index b136e95..b6025cb 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMBTreeIOOperationCallback.java
@@ -39,14 +39,14 @@
     }
 
     @Override
-    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
-        if (oldComponents == null) {
+    public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents == null) {
             // Implies a flush IO operation.
             return opTracker.getLastLSN();
         }
-        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
-        for (ILSMComponent c : oldComponents) {
+        for (ILSMComponent c : diskComponents) {
             BTree btree = ((LSMBTreeDiskComponent) c).getBTree();
             maxLSN = Math.max(getTreeIndexLSN(btree), maxLSN);
         }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
index 655ace2..4f99ae6 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMInvertedIndexIOOperationCallback.java
@@ -38,14 +38,14 @@
     }
 
     @Override
-    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
-        if (oldComponents == null) {
+    public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents == null) {
             // Implies a flush IO operation.
             return opTracker.getLastLSN();
         }
-        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
-        for (Object o : oldComponents) {
+        for (Object o : diskComponents) {
             LSMInvertedIndexDiskComponent invIndexComponent = (LSMInvertedIndexDiskComponent) o;
             maxLSN = Math.max(getTreeIndexLSN(invIndexComponent.getDeletedKeysBTree()), maxLSN);
         }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
index a26bb19..cd7b7a0 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/ioopcallbacks/LSMRTreeIOOperationCallback.java
@@ -39,14 +39,14 @@
     }
 
     @Override
-    protected long getComponentLSN(List<ILSMComponent> oldComponents) throws HyracksDataException {
-        if (oldComponents == null) {
+    public long getComponentLSN(List<ILSMComponent> diskComponents) throws HyracksDataException {
+        if (diskComponents == null) {
             // Implies a flush IO operation.
             return opTracker.getLastLSN();
         }
-        // Get max LSN from the oldComponents. Implies a merge IO operation.
+        // Get max LSN from the diskComponents. Implies a merge IO operation or Recovery operation.
         long maxLSN = -1;
-        for (Object o : oldComponents) {
+        for (Object o : diskComponents) {
             LSMRTreeDiskComponent rtreeComponent = (LSMRTreeDiskComponent) o;
             maxLSN = Math.max(getTreeIndexLSN(rtreeComponent.getRTree()), maxLSN);
         }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
deleted file mode 100644
index a10bf08..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/Buffer.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support. This class is not "thread-safe". For performance
- * concerns, it is required for multiple writers to be able to write to the
- * buffer concurrently and that a writer is never blocked by another writer. The
- * users of this class must ensure that two concurrent writers get to write in
- * exclusive areas in the buffer. A reader and writer may or may not conflict
- * with each other. For example, reading of logs during roll back of a
- * transaction t1 does not conflict with writing of logs by another transaction
- * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
- * a flushing the buffer to disk conflicts with a reader reading the buffer.
- * Appropriate locks are taken on the Buffer in the application logic and not
- * directly imposed by synchronized methods.
- */
-
-public class Buffer implements IBuffer {
-
-    ByteBuffer buffer;
-
-    public Buffer(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    protected Buffer() {
-    }
-
-    public void setBuffer(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    @Override
-    public byte getByte(int offset) {
-        return buffer.get(offset);
-    }
-
-    @Override
-    public byte getByte() {
-        return buffer.get();
-    }
-
-    @Override
-    public void getBytes(byte[] bytes, int offset, int size) {
-        System.arraycopy(buffer.array(), offset, bytes, 0, size);
-    }
-
-    @Override
-    public int getSize() {
-        return buffer.capacity();
-    }
-
-    @Override
-    public int readInt() {
-        return buffer.getInt();
-    }
-
-    @Override
-    public int readInt(int offset) {
-        return buffer.getInt(offset);
-    }
-
-    @Override
-    public long readLong(int offset) {
-        return buffer.getLong(offset);
-    }
-
-    @Override
-    public void put(byte b) {
-        buffer.put(b);
-    }
-
-    @Override
-    public void put(int offset, byte b) {
-        buffer.put(offset, b);
-    }
-
-    @Override
-    public void put(byte[] bytes, int start, int length) {
-        buffer.put(bytes, start, length);
-
-    }
-
-    @Override
-    public void put(byte[] bytes) {
-        buffer.put(bytes);
-    }
-
-    @Override
-    public void writeInt(int value) {
-        buffer.putInt(value);
-    }
-
-    @Override
-    public void writeInt(int offset, int value) {
-        buffer.putInt(offset, value);
-
-    }
-
-    @Override
-    public void writeLong(long value) {
-        buffer.putLong(value);
-    }
-
-    @Override
-    public void writeLong(int offset, long value) {
-        buffer.putLong(offset, value);
-
-    }
-
-    @Override
-    public byte[] getArray() {
-        return buffer.array();
-    }
-
-    @Override
-    public void erase() {
-        Arrays.fill(buffer.array(), (byte) 0);
-    }
-
-    @Override
-    public ByteBuffer getByteBuffer() {
-        return buffer;
-    }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
deleted file mode 100644
index fbe9b4c..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileBasedBuffer.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-/**
- * Represent a buffer that is backed by a physical file. Provider custom APIs
- * for accessing a chunk of the underlying file.
- */
-public class FileBasedBuffer extends Buffer implements IFileBasedBuffer {
-
-    private String filePath;
-    private FileChannel fileChannel;
-    private RandomAccessFile raf;
-    private int bufferSize;
-
-    private int bufferLastFlushOffset;
-    private int bufferNextWriteOffset;
-    private final int diskSectorSize;
-
-    private final ReadWriteLock latch;
-    private final AtomicInteger referenceCount;
-
-    public FileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
-        this.filePath = filePath;
-        buffer = ByteBuffer.allocate(bufferSize);
-        raf = new RandomAccessFile(new File(filePath), "rw");
-        fileChannel = raf.getChannel();
-        fileChannel.position(offset);
-        fileChannel.read(buffer);
-        buffer.position(0);
-        this.bufferSize = bufferSize;
-        buffer.limit(bufferSize);
-        bufferLastFlushOffset = 0;
-        bufferNextWriteOffset = 0;
-        this.diskSectorSize = diskSectorSize;
-        latch = new ReentrantReadWriteLock(true);
-        referenceCount = new AtomicInteger(0);
-    }
-
-    public String getFilePath() {
-        return filePath;
-    }
-
-    public void setFilePath(String filePath) {
-        this.filePath = filePath;
-    }
-
-    @Override
-    public int getSize() {
-        return bufferSize;
-    }
-
-    public void clear() {
-        buffer.clear();
-    }
-
-    @Override
-    public void flush() throws IOException {
-        //flush
-        int pos = bufferLastFlushOffset;
-        int limit = (((bufferNextWriteOffset - 1) / diskSectorSize) + 1) * diskSectorSize;
-        buffer.position(pos);
-        buffer.limit(limit);
-        fileChannel.write(buffer);
-        fileChannel.force(false);
-
-        //update variables
-        bufferLastFlushOffset = limit;
-        bufferNextWriteOffset = limit;
-        buffer.limit(bufferSize);
-    }
-
-    @Override
-    public void put(byte b) {
-        buffer.put(b);
-    }
-
-    @Override
-    public void put(int offset, byte b) {
-        buffer.put(offset, b);
-    }
-
-    @Override
-    public void put(byte[] bytes, int start, int length) {
-        buffer.put(bytes, start, length);
-    }
-
-    @Override
-    public void put(byte[] bytes) {
-        buffer.put(bytes);
-    }
-
-    @Override
-    public void writeInt(int value) {
-        buffer.putInt(value);
-    }
-
-    @Override
-    public void writeInt(int index, int value) {
-        buffer.putInt(index, value);
-    }
-
-    @Override
-    public void writeLong(long value) {
-        buffer.putLong(value);
-    }
-
-    @Override
-    public void writeLong(int index, long value) {
-        buffer.putLong(index, value);
-    }
-
-    /**
-     * Resets the buffer with content (size as specified) from a given file
-     * starting at offset.
-     */
-    @Override
-    public void reset(String filePath, long diskNextWriteOffset, int bufferSize) throws IOException {
-        if (!filePath.equals(this.filePath)) {
-            raf.close();//required?
-            fileChannel.close();
-            raf = new RandomAccessFile(filePath, "rw");
-            this.filePath = filePath;
-        }
-        fileChannel = raf.getChannel();
-        fileChannel.position(diskNextWriteOffset);
-        erase();
-        buffer.position(0);
-        buffer.limit(bufferSize);
-        this.bufferSize = bufferSize;
-
-        bufferLastFlushOffset = 0;
-        bufferNextWriteOffset = 0;
-    }
-
-    @Override
-    public void close() throws IOException {
-        fileChannel.close();
-    }
-
-    @Override
-    public void open(String filePath, long offset, int bufferSize) throws IOException {
-        raf = new RandomAccessFile(filePath, "rw");
-        fileChannel = raf.getChannel();
-        fileChannel.position(offset);
-        erase();
-        buffer.position(0);
-        buffer.limit(bufferSize);
-        this.bufferSize = bufferSize;
-        bufferLastFlushOffset = 0;
-        bufferNextWriteOffset = 0;
-    }
-
-    @Override
-    public long getDiskNextWriteOffset() throws IOException {
-        return fileChannel.position();
-    }
-
-    @Override
-    public void setDiskNextWriteOffset(long offset) throws IOException {
-        fileChannel.position(offset);
-    }
-
-    @Override
-    public int getBufferLastFlushOffset() {
-        return bufferLastFlushOffset;
-    }
-
-    @Override
-    public void setBufferLastFlushOffset(int offset) {
-        this.bufferLastFlushOffset = offset;
-    }
-
-    @Override
-    public int getBufferNextWriteOffset() {
-        synchronized (fileChannel) {
-            return bufferNextWriteOffset;
-        }
-    }
-
-    @Override
-    public void setBufferNextWriteOffset(int offset) {
-        synchronized (fileChannel) {
-            if (bufferNextWriteOffset < offset) {
-                bufferNextWriteOffset = offset;
-            }
-        }
-    }
-
-    @Override
-    public void acquireWriteLatch() {
-        latch.writeLock().lock();
-    }
-
-    @Override
-    public void releaseWriteLatch() {
-        latch.writeLock().unlock();
-    }
-
-    @Override
-    public void acquireReadLatch() {
-        latch.readLock().lock();
-    }
-
-    @Override
-    public void releaseReadLatch() {
-        latch.readLock().unlock();
-    }
-
-    @Override
-    public void incRefCnt() {
-        referenceCount.incrementAndGet();
-    }
-    
-    @Override
-    public void decRefCnt() {
-        referenceCount.decrementAndGet();
-    }
-    
-    @Override
-    public int getRefCnt() {
-        return referenceCount.get();
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
deleted file mode 100644
index 64beb86..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/FileUtil.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.io.File;
-import java.io.IOException;
-
-/**
- * A utility class that provides operations on files such as creation and
- * loading content in a buffer. It also provides API for creating directories
- */
-public class FileUtil {
-
-    public static final String lineSeparator = System.getProperty("line.separator");
-
-    public static boolean createFileIfNotExists(String path) throws IOException {
-        File file = new File(path);
-        File parentFile = file.getParentFile();
-        if (parentFile != null) {
-            parentFile.mkdirs();
-        }
-        return file.createNewFile();
-    }
-
-    public static boolean createNewDirectory(String path) throws IOException {
-        return (new File(path)).mkdir();
-    }
-
-    public static IFileBasedBuffer getFileBasedBuffer(String filePath, long offset, int bufferSize, int diskSectorSize) throws IOException {
-        IFileBasedBuffer fileBasedBuffer = new FileBasedBuffer(filePath, offset, bufferSize, diskSectorSize);
-        return fileBasedBuffer;
-    }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
deleted file mode 100644
index fd4af81..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IBuffer.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support.
- */
-public interface IBuffer {
-
-    public byte getByte(int offset);
-
-    public int getSize();
-
-    public byte getByte();
-
-    public void getBytes(byte[] bytes, int start, int size);
-
-    public int readInt();
-
-    public int readInt(int offset);
-
-    public long readLong(int offset);
-
-    public void put(byte b);
-
-    public void put(int offset, byte b);
-
-    public void put(byte[] bytes, int start, int length);
-
-    public void put(byte[] bytes);
-
-    public void writeInt(int value);
-
-    public void writeLong(long value);
-
-    public void writeInt(int offset, int value);
-
-    public void writeLong(int offset, long value);
-
-    public byte[] getArray();
-
-    public void erase();
-
-    public ByteBuffer getByteBuffer();
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
deleted file mode 100644
index 1c8cb76..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ICloseable.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * Represents a closeable resource that implements a close(@see
- * TransactionContext) method. When a transaction commits/aborts, the close
- * method is called on each of the ICloseable resources that were involved in
- * the transaction. This gives an opportunity to release all resources and do a
- * cleanup. An example of ICloseable is the @see TreeLogger.
- */
-public interface ICloseable {
-
-    /**
-     * This method is invoked at the commit/abort of a transaction that involved
-     * a ICloseable resource. It is used to do a clean up by the involved
-     * resource before the transaction ends.
-     * 
-     * @param context
-     * @throws ACIDException
-     */
-    public void close(ITransactionContext context) throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
deleted file mode 100644
index 1d466e3..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IFileBasedBuffer.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.io.IOException;
-
-/**
- * Represent a buffer that is backed by a physical file. Provides custom APIs
- * for accessing a chunk of the underlying file.
- */
-
-public interface IFileBasedBuffer extends IBuffer {
-
-    public void flush() throws IOException;
-
-    /**
-     * Resets the buffer with content (size as specified) from a given file
-     * starting at offset.
-     */
-    public void reset(String filePath, long offset, int size) throws IOException;
-
-    public long getDiskNextWriteOffset() throws IOException;
-
-    public void setDiskNextWriteOffset(long writePosition) throws IOException;
-
-    public void close() throws IOException;
-    
-    public void open(String filePath, long offset, int size) throws IOException;
-
-    public int getBufferLastFlushOffset();
-
-    public void setBufferLastFlushOffset(int offset);
-
-    public int getBufferNextWriteOffset();
-
-    public void setBufferNextWriteOffset(int offset);
-    
-    public void acquireWriteLatch();
-
-    public void releaseWriteLatch();
-
-    public void acquireReadLatch();
-
-    public void releaseReadLatch();
-    
-    public void incRefCnt();
-    
-    public void decRefCnt();
-    
-    public int getRefCnt();
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
index 698a788..54c86af 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILockManager.java
@@ -64,9 +64,9 @@
      * @param txnContext
      * @throws ACIDException
      *             TODO
-     * @return true if the lock count is 0, false otherwise.
+     * @return 
      */
-    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
+    public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
             throws ACIDException;
 
     /**
@@ -75,9 +75,9 @@
      * @param txnContext
      * @throws ACIDException
      *             TODO
-     * @return true if the lock count is 0, false otherwise.
+     * @return 
      */
-    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
+    public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
             throws ACIDException;
 
     /**
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
deleted file mode 100644
index 5c5c149..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogFilter.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * Represents a predicate or a filter that needs to be applied when selectively
- * retrieving logs.
- */
-public interface ILogFilter {
-
-    public boolean accept(IBuffer buffer, long startOffset, int length);
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
index 9cc4d0d..8913f8a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogManager.java
@@ -14,73 +14,10 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
 public interface ILogManager {
 
-    /**
-     * @param logType
-     * @param context
-     * @param datasetId
-     * @param PKHashValue
-     * @param resourceId
-     * @param resourceMgrId
-     * @param logContentSize
-     * @param reusableLogContentObject
-     * @param logger
-     * @param logicalLogLocator
-     * @throws ACIDException
-     */
-    void log(byte logType, ITransactionContext context, int datasetId, int PKHashValue, long resourceId,
-            byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
-            LogicalLogLocator logicalLogLocator) throws ACIDException;
+    public void log(ILogRecord logRecord);
 
-    /**
-     * @param physicalLogLocator
-     *            specifies the physical location from where the logs need to be
-     *            read
-     * @param logFilter
-     *            specifies the filtering criteria for the retrieved logs
-     * @return LogCursor an iterator for the retrieved logs
-     * @throws ACIDException
-     */
-    public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
-            ACIDException;
-
-    /**
-     * @param lsnValue
-     *            TODO
-     * @param logicalLogLocator
-     *            TODO
-     * @throws ACIDException
-     */
-    public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException;
-
-    /**
-     * Retrieves the configuration parameters of the ILogManager
-     * 
-     * @return LogManagerProperties: the configuration parameters for the
-     *         ILogManager
-     */
-    public LogManagerProperties getLogManagerProperties();
-
-    /**
-     * Returns the ILogRecordHelper instance associated with this ILogManager
-     * instance
-     * 
-     * @return ILogRecordHelper: the utility (class) for writing/reading log
-     *         header.
-     */
-    public ILogRecordHelper getLogRecordHelper();
-
-    /**
-     * Returns the Transaction Provider associated with this ILogManager
-     * instance
-     * 
-     * @return TransactionSubsystem
-     */
-    public ITransactionSubsystem getTransactionSubsystem();
+    public ILogReader getLogReader(boolean isRecoveryMode);
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
similarity index 78%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
index 08940a7..7d9bba7 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILoggerRepository.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogPage.java
@@ -14,9 +14,10 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
+public interface ILogPage {
 
-public interface ILoggerRepository {
+    public void append(ILogRecord logRecord, long appendLsn);
 
-    public ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException;
-}
+    public void flush() throws InterruptedException;
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
similarity index 70%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
index e455d3cc..fa92865 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogCursor.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogReader.java
@@ -14,18 +14,18 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-import java.io.IOException;
-
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
 
-/**
- * Provides a cursor over the logs created to date.
- */
-public interface ILogCursor {
+public interface ILogReader {
 
-    public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException;
+    public void initializeScan(long beginLSN) throws ACIDException;
 
-    public ILogFilter getLogFilter();
+    //for scanning
+    public ILogRecord next() throws ACIDException;
 
-}
+    //for random reading
+    public ILogRecord read(long readLSN) throws ACIDException;
+
+    public void close() throws ACIDException;
+
+}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
new file mode 100644
index 0000000..d810ebd
--- /dev/null
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecord.java
@@ -0,0 +1,108 @@
+/*
+ * Copyright 2009-2013 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.common.transactions;
+
+import java.nio.ByteBuffer;
+
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+
+public interface ILogRecord {
+
+    public static final int COMMIT_LOG_SIZE = 21;
+    public static final int UPDATE_LOG_BASE_SIZE = 56;
+
+    public boolean readLogRecord(ByteBuffer buffer);
+
+    public void writeLogRecord(ByteBuffer buffer);
+    
+    public void formCommitLogRecord(ITransactionContext txnCtx, byte logType, int jobId, int datasetId, int PKHashValue);
+
+    public void setUpdateLogSize();
+
+    public ITransactionContext getTxnCtx();
+
+    public void setTxnCtx(ITransactionContext txnCtx);
+
+    public boolean isFlushed();
+
+    public void isFlushed(boolean isFlushed);
+
+    public byte getLogType();
+
+    public void setLogType(byte logType);
+
+    public int getJobId();
+
+    public void setJobId(int jobId);
+
+    public int getDatasetId();
+
+    public void setDatasetId(int datasetId);
+
+    public int getPKHashValue();
+
+    public void setPKHashValue(int PKHashValue);
+
+    public long getPrevLSN();
+
+    public void setPrevLSN(long prevLsn);
+
+    public long getResourceId();
+
+    public void setResourceId(long resourceId);
+
+    public byte getResourceType();
+
+    public void setResourceType(byte resourceType);
+
+    public int getLogSize();
+
+    public void setLogSize(int logSize);
+
+    public byte getNewOp();
+
+    public void setNewOp(byte newOp);
+
+    public int getNewValueSize();
+
+    public void setNewValueSize(int newValueSize);
+
+    public ITupleReference getNewValue();
+
+    public void setNewValue(ITupleReference newValue);
+
+    public byte getOldOp();
+
+    public void setOldOp(byte oldOp);
+
+    public int getOldValueSize();
+
+    public void setOldValueSize(int oldValueSize);
+
+    public ITupleReference getOldValue();
+
+    public void setOldValue(ITupleReference oldValue);
+
+    public long getChecksum();
+
+    public void setChecksum(long checksum);
+    
+    public long getLSN();
+
+    public void setLSN(long LSN);
+
+    public String getLogRecordForDisplay();
+
+}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
deleted file mode 100644
index a299a0d..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogRecordHelper.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-/**
- * Helper class for writing/reading of log header and checksum as well as
- * validating log record by checksum comparison. Every ILogManager
- * implementation has an associated ILogRecordHelper implementation.
- */
-
-public interface ILogRecordHelper {
-
-    public byte getLogType(LogicalLogLocator logicalLogLocator);
-
-    public int getJobId(LogicalLogLocator logicalLogLocator);
-
-    public int getDatasetId(LogicalLogLocator logicalLogLocator);
-
-    public int getPKHashValue(LogicalLogLocator logicalLogLocator);
-
-    public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator);
-
-    public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator);
-
-    public long getResourceId(LogicalLogLocator logicalLogLocator);
-
-    public byte getResourceMgrId(LogicalLogLocator logicalLogLocater);
-
-    public int getLogContentSize(LogicalLogLocator logicalLogLocater);
-
-    public long getLogChecksum(LogicalLogLocator logicalLogLocator);
-
-    public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator);
-
-    public int getLogContentEndPos(LogicalLogLocator logicalLogLocator);
-
-    public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator);
-
-    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
-            int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
-            int logRecordSize);
-
-    public boolean validateLogRecord(LogicalLogLocator logicalLogLocator);
-
-    public int getLogRecordSize(byte logType, int logBodySize);
-
-    public int getLogHeaderSize(byte logType);
-
-    public int getLogChecksumSize();
-
-    public int getCommitLogSize();
-
-}
\ No newline at end of file
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
deleted file mode 100644
index 939f8e5..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ILogger.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * An interface providing call back APIs that are invoked {@link ILogManager} for providing the content for the log record and doing any pre/post
- * processing.
- */
-public interface ILogger {
-
-    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
-            throws ACIDException;
-
-    public void log(ITransactionContext context, final LogicalLogLocator logicalLogLocator, int logContentSize,
-            ReusableLogContentObject reusableLogContentObject) throws ACIDException;
-
-    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
-            throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
index 21a02ce..6d2f3cb 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IRecoveryManager.java
@@ -33,6 +33,12 @@
         HEALTHY,
         CORRUPTED
     }
+    
+    public class ResourceType {
+        public static final byte LSM_BTREE = 0;
+        public static final byte LSM_RTREE = 1;
+        public static final byte LSM_INVERTED_INDEX = 2;
+    }
 
     /**
      * Returns the state of the system.
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
deleted file mode 100644
index 36ce317..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/IResourceManager.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-
-/**
- * Provides APIs for undo or redo of an operation on a resource.
- */
-public interface IResourceManager {
-    
-    public class ResourceType {
-        public static final byte LSM_BTREE = 1;
-        public static final byte LSM_RTREE = 2;
-        public static final byte LSM_INVERTED_INDEX = 3;
-    }
-
-    /**
-     * Returns the unique identifier for the resource manager.
-     * 
-     * @return a unique identifier for the resource manager. The number of
-     *         resource managers in the system are expected to be handful and
-     *         can be uniquely identified by using a single byte as an id.
-     */
-    public byte getResourceManagerId();
-
-    /**
-     * Undo the operation corresponding to a given log record.
-     * 
-     * @param logRecordHelper
-     *            (@see ILogRecordHelper) An implementation of the
-     *            ILogRecordHelper interface that is used to parse the log
-     *            record and extract useful information from the content.
-     * @param LogicalLogLocator
-     *            (@see LogicalLogLocator) The locationof the log record that
-     *            needs to be undone.
-     * @throws ACIDException
-     */
-    public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logicalLogLocator) throws ACIDException;
-
-    /**
-     * Redo the operation corresponding to a given log record.
-     * 
-     * @param logRecordHelper
-     *            (@see ILogRecordHelper) An implementation of the
-     *            ILogRecordHelper interface that is used to parse the log
-     *            record and extract useful information from the content.
-     * @param LogicalLogLocator
-     *            (@see LogicalLogLocator) The locationof the log record that
-     *            needs to be undone.
-     * @throws ACIDException
-     */
-    public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException;
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
index 31f3e64..b9ad1bd 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionContext.java
@@ -14,24 +14,13 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 
 public interface ITransactionContext {
 
-    public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback);
-
-    public void updateLastLSNForIndexes(long lastLSN);
-
-    public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException;
-
-    public LogicalLogLocator getFirstLogLocator();
-
-    public LogicalLogLocator getLastLogLocator();
-
-    public void addCloseableResource(ICloseable resource);
+    public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
+            boolean isPrimaryIndex);
 
     public JobId getJobId();
 
@@ -47,9 +36,11 @@
 
     public TransactionState getTxnState();
 
-    public void releaseResources() throws ACIDException;
+    public long getFirstLSN();
 
-    public void setLastLSN(long lsn);
+    public long getLastLSN();
+
+    public void setLastLSN(long resourceId, long LSN);
 
     public TransactionType getTransactionType();
 
@@ -57,8 +48,9 @@
 
     public String prettyPrint();
 
-    public static final long INVALID_TIME = -1l; // used for showing a
-    // transaction is not waiting.
+    // used for showing a transaction is not waiting.
+    public static final long INVALID_TIME = -1l;
+
     public static final int ACTIVE_STATUS = 0;
     public static final int TIMED_OUT_STATUS = 1;
 
@@ -68,7 +60,7 @@
     }
 
     public void setExclusiveJobLevelCommit();
-    
-    public boolean isExlusiveJobLevelCommit();
+
+    void notifyOptracker(boolean isJobLevelCommit);
 
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
index 63ee5d0..b6c934a 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ITransactionSubsystem.java
@@ -14,7 +14,6 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-
 public interface ITransactionSubsystem {
 
     public ILogManager getLogManager();
@@ -25,10 +24,6 @@
 
     public IRecoveryManager getRecoveryManager();
 
-    public TransactionalResourceManagerRepository getTransactionalResourceRepository();
-
-    public ILoggerRepository getTreeLoggerRepository();
-
     public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider();
 
     public String getId();
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
index 4b5eb9b..591d9b1 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogManagerProperties.java
@@ -22,6 +22,7 @@
 
     private static final long serialVersionUID = 2084227360840799662L;
 
+    public static final String lineSeparator = System.getProperty("line.separator");
     public static final int LOG_MAGIC_NUMBER = 123456789;
     public static final String LOG_DIR_SUFFIX = ".txnLogDir";
     private static final String DEFAULT_LOG_FILE_PREFIX = "asterix_transaction_log";
@@ -97,13 +98,12 @@
 
     public String toString() {
         StringBuilder builder = new StringBuilder();
-        builder.append("log_dir_ : " + logDir + FileUtil.lineSeparator);
-        builder.append("log_file_prefix" + logFilePrefix + FileUtil.lineSeparator);
-        builder.append("log_page_size : " + logPageSize + FileUtil.lineSeparator);
-        builder.append("num_log_pages : " + numLogPages + FileUtil.lineSeparator);
-        builder.append("log_partition_size : " + logPartitionSize + FileUtil.lineSeparator);
-        builder.append("group_commit_wait_period : " + groupCommitWaitPeriod + FileUtil.lineSeparator);
-        builder.append("disk_sector_size : " + diskSectorSize + FileUtil.lineSeparator);
+        builder.append("log_dir_ : " + logDir + lineSeparator);
+        builder.append("log_file_prefix" + logFilePrefix + lineSeparator);
+        builder.append("log_page_size : " + logPageSize + lineSeparator);
+        builder.append("num_log_pages : " + numLogPages + lineSeparator);
+        builder.append("log_partition_size : " + logPartitionSize + lineSeparator);
+        builder.append("group_commit_wait_period : " + groupCommitWaitPeriod + lineSeparator);
         return builder.toString();
     }
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
deleted file mode 100644
index 58fc62a..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogUtil.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.io.File;
-import java.io.FilenameFilter;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.logging.Level;
-import java.util.logging.Logger;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-/**
- * A utility class providing helper methods for the {@link ILogManager}
- */
-public class LogUtil {
-
-    private static final Logger LOGGER = Logger.getLogger(LogUtil.class.getName());
-
-    // read the log directory and initialize log anchor to point to the
-    // current log partition file and the offset where the log manager shall
-    // continue to insert log records.
-
-    public static PhysicalLogLocator initializeLogAnchor(ILogManager logManager) throws ACIDException {
-        int fileId = 0;
-        long offset = 0;
-        LogManagerProperties logManagerProperties = logManager.getLogManagerProperties();
-        File logDir = new File(logManagerProperties.getLogDir());
-        try {
-            if (logDir.exists()) {
-                List<String> logFiles = getLogFiles(logManagerProperties);
-                if (logFiles == null || logFiles.size() == 0) {
-                    FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
-                    if (LOGGER.isLoggable(Level.INFO)) {
-                        LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
-                    }
-                } else {
-                    File logFile = new File(LogUtil.getLogFilePath(logManagerProperties,
-                            Long.parseLong(logFiles.get(logFiles.size() - 1))));
-                    fileId = logFiles.size() - 1;
-                    offset = logFile.length();
-                }
-            } else {
-                FileUtil.createNewDirectory(logManagerProperties.getLogDir());
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
-                }
-                FileUtil.createFileIfNotExists(getLogFilePath(logManagerProperties, 0));
-                if (LOGGER.isLoggable(Level.INFO)) {
-                    LOGGER.info("created a log file: " + getLogFilePath(logManagerProperties, 0));
-                }
-            }
-        } catch (IOException ioe) {
-            throw new ACIDException("Unable to initialize log anchor", ioe);
-        }
-        if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" file id :" + fileId + " offset " + offset);
-        }
-        return new PhysicalLogLocator(fileId, offset, logManager);
-    }
-
-    public static List<String> getLogFiles(final LogManagerProperties logManagerProperties) {
-        File logDir = new File(logManagerProperties.getLogDir());
-        String[] logFiles = new String[0];
-        List<String> logFileCollection = new ArrayList<String>();
-        if (logDir.exists()) {
-            logFiles = logDir.list(new FilenameFilter() {
-                public boolean accept(File dir, String name) {
-                    if (name.startsWith(logManagerProperties.getLogFilePrefix())) {
-                        return true;
-                    }
-                    return false;
-                }
-            });
-        }
-        for (String logFile : logFiles) {
-            logFileCollection.add(logFile.substring(logManagerProperties.getLogFilePrefix().length() + 1));
-        }
-        Collections.sort(logFileCollection, new Comparator<String>() {
-            @Override
-            public int compare(String arg0, String arg1) {
-                return Integer.parseInt(arg0) - Integer.parseInt(arg1);
-            }
-        });
-        return logFileCollection;
-    }
-
-    public static long getFileId(String logFilePath, LogManagerProperties logManagerProperties) {
-        String logFileName = logFilePath;
-        if (logFilePath.contains(File.separator)) {
-            logFileName = logFilePath.substring(logFilePath.lastIndexOf(File.separator));
-        }
-        return Long.parseLong(logFileName.substring(logFileName.indexOf(logManagerProperties.getLogFilePrefix())));
-    }
-
-    public static String getLogFilePath(LogManagerProperties logManagerProperties, long fileId) {
-        return logManagerProperties.getLogDir() + File.separator + logManagerProperties.getLogFilePrefix() + "_"
-                + fileId;
-    }
-
-    public static LogicalLogLocator getDummyLogicalLogLocator(ILogManager logManager) {
-        LogicalLogLocator logicalLogLocator = new LogicalLogLocator(-1, null, -1, logManager);
-        return logicalLogLocator;
-    }
-
-    /*
-     * given a lsn, get the offset within the log file where the corresponding
-     * log record is (to be) placed.
-     */
-    public static long getFileOffset(ILogManager logManager, long lsn) {
-        return lsn % logManager.getLogManagerProperties().getLogPartitionSize();
-    }
-
-    /*
-     * given a lsn, get the file id that contains the log record.
-     */
-    public static long getFileId(ILogManager logManager, long lsn) {
-        return lsn / logManager.getLogManagerProperties().getLogPartitionSize();
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
deleted file mode 100644
index 9ba7682..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/LogicalLogLocator.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Represents a location of a log record. The location has two parts to it. A
- * LogicalLogLocator extends PhyscialLogLocator and hence can also be used to
- * determine the physical location of the log record on the local filesystem. In
- * addition to the physical location, a LogicalLogLocator also contains a handle
- * to an in-memory buffer and an offset within the buffer where the log record
- * resides.
- */
-public class LogicalLogLocator extends PhysicalLogLocator {
-
-    private IBuffer buffer;
-    private AtomicInteger memoryOffset;
-
-    public LogicalLogLocator(long lsnValue, IBuffer buffer, int bufferOffset, ILogManager logManager) {
-        super(lsnValue, logManager);
-        this.buffer = buffer;
-        this.memoryOffset = new AtomicInteger(bufferOffset);
-
-    }
-
-    public IBuffer getBuffer() {
-        return buffer;
-    }
-
-    public void setBuffer(IBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    public int getMemoryOffset() {
-        return memoryOffset.get();
-    }
-
-    public void setMemoryOffset(int memoryOffset) {
-        this.memoryOffset.set(memoryOffset);
-    }
-
-    @Override
-    public String toString() {
-        return super.toString() + " " + "memoryOffset:" + memoryOffset;
-    }
-
-    public boolean checkValidity() {
-        return true;
-    }
-
-    public long increaseMemoryOffset(int delta) {
-        return memoryOffset.addAndGet(delta);
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
similarity index 66%
rename from asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
rename to asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
index fb865aa..4480aba 100644
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableResourceId.java
+++ b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/MutableLong.java
@@ -14,31 +14,39 @@
  */
 package edu.uci.ics.asterix.common.transactions;
 
-public class MutableResourceId{
-    long id;
+public class MutableLong {
+    private long val;
 
-    public MutableResourceId(long id) {
-        this.id = id;
+    
+    public MutableLong() {
+    }
+    
+    public MutableLong(long val) {
+        this.val = val;
+    }
+    
+    public void set(long val) {
+        this.val = val;
     }
 
-    public void setId(long id) {
-        this.id = id;
+    public long get() {
+        return val;
     }
-
-    public long getId() {
-        return id;
-    }
-
+    
     @Override
     public int hashCode() {
-        return (int)id;
+        return (int)val;
     }
 
     @Override
     public boolean equals(Object o) {
-        if ((o == null) || !(o instanceof MutableResourceId)) {
+        if (o == this) {
+            return true;
+        }
+        if (!(o instanceof MutableLong)) {
             return false;
         }
-        return ((MutableResourceId) o).id == this.id;
+        return ((MutableLong) o).val == val;
     }
+
 }
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
deleted file mode 100644
index b83a3f3..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/PhysicalLogLocator.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * Represents the physical location of a log record. The physical location i
- * deciphered from the contained lsn that is broken down into a file id and an
- * offset within the file. The mapping between fileId and the path on the local
- * file system is maintained by the log manager (@see ILogManager) path on the
- * local file system.
- */
-public class PhysicalLogLocator {
-
-    // The log sequence number corresponding to the log record that is being
-    // referred to.
-    private final AtomicLong lsn;
-
-    // Handle to the log manager that wrote the log record.
-    private final ILogManager logManager;
-
-    public static long getLsnValue(long fileId, long offset, ILogManager logManager) {
-        return fileId * logManager.getLogManagerProperties().getLogPartitionSize() + offset;
-    }
-
-    public PhysicalLogLocator(long lsn, ILogManager logManager) {
-        this.lsn = new AtomicLong(lsn);
-        this.logManager = logManager;
-    }
-
-    public PhysicalLogLocator(long fileId, long offset, ILogManager logManager) {
-        this.lsn = new AtomicLong(getLsnValue(fileId, offset, logManager));
-        this.logManager = logManager;
-    }
-
-    @Override
-    public String toString() {
-        return "lsn :" + lsn.get();
-    }
-
-    public long getFileId() {
-        return LogUtil.getFileId(logManager, lsn.get());
-    }
-
-    public boolean compareAndSet(long expect, long update) {
-        return lsn.compareAndSet(expect, update);
-    }
-
-    public long getFileOffset() {
-        return LogUtil.getFileOffset(logManager, lsn.get());
-    }
-
-    public long getLsn() {
-        return lsn.get();
-    }
-
-    public long incrementLsn() {
-        return lsn.incrementAndGet();
-    }
-
-    public long incrementLsn(long delta) {
-        return lsn.addAndGet(delta);
-    }
-
-    public void setLsn(long lsn) {
-        this.lsn.set(lsn);
-    }
-
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
deleted file mode 100644
index b513fad..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/ReusableLogContentObject.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-
-/**
- * Represents the state of a transaction thread. The state contains information
- * that includes the tuple being operated, the operation and the location of the
- * log record corresponding to the operation.
- */
-public class ReusableLogContentObject {
-
-    private LogicalLogLocator logicalLogLocator;
-    private IndexOperation newOperation;
-    private ITupleReference newValue;
-    private IndexOperation oldOperation;
-    private ITupleReference oldValue;
-
-    public ReusableLogContentObject(LogicalLogLocator logicalLogLocator, IndexOperation newOperation,
-            ITupleReference newValue, IndexOperation oldOperation, ITupleReference oldValue) {
-        this.logicalLogLocator = logicalLogLocator;
-        this.newOperation = newOperation;
-        this.newValue = newValue;
-        this.oldOperation = oldOperation;
-        this.oldValue = oldValue;
-    }
-
-    public synchronized LogicalLogLocator getLogicalLogLocator() {
-        return logicalLogLocator;
-    }
-
-    public synchronized void setLogicalLogLocator(LogicalLogLocator logicalLogLocator) {
-        this.logicalLogLocator = logicalLogLocator;
-    }
-
-    public synchronized void setNewOperation(IndexOperation newOperation) {
-        this.newOperation = newOperation;
-    }
-
-    public synchronized IndexOperation getNewOperation() {
-        return newOperation;
-    }
-
-    public synchronized void setNewValue(ITupleReference newValue) {
-        this.newValue = newValue;
-    }
-
-    public synchronized ITupleReference getNewValue() {
-        return newValue;
-    }
-
-    public synchronized void setOldOperation(IndexOperation oldOperation) {
-        this.oldOperation = oldOperation;
-    }
-
-    public synchronized IndexOperation getOldOperation() {
-        return oldOperation;
-    }
-
-    public synchronized void setOldValue(ITupleReference oldValue) {
-        this.oldValue = oldValue;
-    }
-
-    public synchronized ITupleReference getOldValue() {
-        return oldValue;
-    }
-}
diff --git a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java b/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
deleted file mode 100644
index 64c90f4..0000000
--- a/asterix-common/src/main/java/edu/uci/ics/asterix/common/transactions/TransactionalResourceManagerRepository.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Copyright 2009-2013 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.common.transactions;
-
-import java.util.HashMap;
-import java.util.Map;
-
-/**
- * Represents a repository containing Resource Managers and Resources in the
- * transaction eco-system. Operations on a resource require acquiring
- * appropriate locks (for isolation) and writing logs (durability). Every
- * resource is managed by an associated resource manager that contains the logic
- * to interpret the logs and take necessary action(s) during roll back or
- * recovery. An example of resource is a @see ITreeIndex that is managed by a
- * resource manager @see TreeResourceManager
- */
-public class TransactionalResourceManagerRepository {
-
-    private Map<Byte, IResourceManager> resourceMgrRepository = new HashMap<Byte, IResourceManager>(); // repository
-
-    public void registerTransactionalResourceManager(byte id, IResourceManager resourceMgr) {
-        synchronized (resourceMgrRepository) {
-            if (resourceMgrRepository.get(id) == null) {
-                resourceMgrRepository.put(id, resourceMgr);
-            }
-        }
-    }
-
-    public IResourceManager getTransactionalResourceMgr(byte id) {
-        synchronized (resourceMgrRepository) {
-            return resourceMgrRepository.get(id);
-        }
-    }
-}
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
index f742a64..edffcd7 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/MetadataNode.java
@@ -27,7 +27,7 @@
 import edu.uci.ics.asterix.common.functions.FunctionSignature;
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext.TransactionType;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
@@ -109,7 +109,8 @@
 
     @Override
     public void beginTransaction(JobId transactionId) throws ACIDException, RemoteException {
-        transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
+        ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().beginTransaction(transactionId);
+        txnCtx.setExclusiveJobLevelCommit();
     }
 
     @Override
@@ -275,7 +276,8 @@
 
         ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
         txnCtx.setTransactionType(TransactionType.READ_WRITE);
-        txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+        txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
+                metadataIndex.isPrimaryIndex());
 
         // TODO: fix exceptions once new BTree exception model is in hyracks.
         indexAccessor.insert(tuple);
@@ -582,7 +584,8 @@
 
         ITransactionContext txnCtx = transactionSubsystem.getTransactionManager().getTransactionContext(jobId);
         txnCtx.setTransactionType(TransactionType.READ_WRITE);
-        txnCtx.registerIndexAndCallback(lsmIndex, (AbstractOperationCallback) modCallback);
+        txnCtx.registerIndexAndCallback(resourceID, lsmIndex, (AbstractOperationCallback) modCallback,
+                metadataIndex.isPrimaryIndex());
 
         indexAccessor.delete(tuple);
         indexLifecycleManager.close(resourceID);
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
index 4f67cb5..ca5e56c 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/bootstrap/MetadataBootstrap.java
@@ -36,8 +36,6 @@
 import edu.uci.ics.asterix.common.context.DatasetLifecycleManager;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMBTreeIOOperationCallbackFactory;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
 import edu.uci.ics.asterix.external.adapter.factory.IAdapterFactory;
 import edu.uci.ics.asterix.external.dataset.adapter.AdapterIdentifier;
 import edu.uci.ics.asterix.metadata.IDatasetDetails;
@@ -62,7 +60,6 @@
 import edu.uci.ics.asterix.runtime.formats.NonTaggedDataFormat;
 import edu.uci.ics.asterix.transaction.management.resource.LSMBTreeLocalResourceMetadata;
 import edu.uci.ics.asterix.transaction.management.resource.PersistentLocalResourceFactoryProvider;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
 import edu.uci.ics.asterix.transaction.management.service.transaction.AsterixRuntimeComponentsProvider;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.application.INCApplicationContext;
@@ -139,15 +136,6 @@
         MetadataSecondaryIndexes.init();
         initLocalIndexArrays();
 
-        TransactionalResourceManagerRepository resourceRepository = runtimeContext.getTransactionSubsystem()
-                .getTransactionalResourceRepository();
-        resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_BTREE, new IndexResourceManager(
-                ResourceType.LSM_BTREE, runtimeContext.getTransactionSubsystem()));
-        resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_RTREE, new IndexResourceManager(
-                ResourceType.LSM_RTREE, runtimeContext.getTransactionSubsystem()));
-        resourceRepository.registerTransactionalResourceManager(ResourceType.LSM_INVERTED_INDEX,
-                new IndexResourceManager(ResourceType.LSM_INVERTED_INDEX, runtimeContext.getTransactionSubsystem()));
-
         AsterixMetadataProperties metadataProperties = propertiesProvider.getMetadataProperties();
         metadataNodeName = metadataProperties.getMetadataNodeName();
         metadataStore = metadataProperties.getMetadataStore();
diff --git a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
index a5b6523..8b422c8 100644
--- a/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
+++ b/asterix-metadata/src/main/java/edu/uci/ics/asterix/metadata/declared/AqlMetadataProvider.java
@@ -37,7 +37,7 @@
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMInvertedIndexIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.ioopcallbacks.LSMRTreeIOOperationCallbackFactory;
 import edu.uci.ics.asterix.common.parse.IParseFileSplitsDecl;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.JobId;
 import edu.uci.ics.asterix.dataflow.data.nontagged.valueproviders.AqlPrimitiveValueProviderFactory;
 import edu.uci.ics.asterix.external.adapter.factory.HDFSAdapterFactory;
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
new file mode 100644
index 0000000..dac3e95
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/AbstractIndexModificationOperationCallback.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2009-2013 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.transaction.management.opcallbacks;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
+import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+
+public abstract class AbstractIndexModificationOperationCallback extends AbstractOperationCallback {
+
+    protected final long resourceId;
+    protected final byte resourceType;
+    protected final IndexOperation indexOp;
+    protected final ITransactionSubsystem txnSubsystem;
+    protected final SimpleTupleWriter tupleWriter;
+    protected final ILogRecord logRecord;
+
+    protected AbstractIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
+            ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
+            byte resourceType, IndexOperation indexOp) {
+        super(datasetId, primaryKeyFields, txnCtx, lockManager);
+        this.resourceId = resourceId;
+        this.resourceType = resourceType;
+        this.indexOp = indexOp;
+        this.txnSubsystem = txnSubsystem;
+        tupleWriter = new SimpleTupleWriter();
+        logRecord = new LogRecord();
+        logRecord.setTxnCtx(txnCtx);
+        logRecord.setLogType(LogType.UPDATE);
+        logRecord.setJobId(txnCtx.getJobId().getId());
+        logRecord.setDatasetId(datasetId);
+        logRecord.setResourceId(resourceId);
+        logRecord.setResourceType(resourceType);
+        logRecord.setNewOp((byte) (indexOp.ordinal()));
+    }
+
+    protected void log(int PKHash, ITupleReference newValue, IndexOperation oldOp, ITupleReference oldValue)
+            throws ACIDException {
+        logRecord.setPKHashValue(PKHash);
+        if (newValue != null) {
+            logRecord.setNewValueSize(tupleWriter.bytesRequired(newValue));
+            logRecord.setNewValue(newValue);
+        } else {
+            logRecord.setNewValueSize(0);
+        }
+        if (resourceType == ResourceType.LSM_BTREE) {
+            logRecord.setOldOp((byte) (oldOp.ordinal()));
+            if (oldValue != null) {
+                logRecord.setOldValueSize(tupleWriter.bytesRequired(oldValue));
+                logRecord.setOldValue(oldValue);
+            } else {
+                logRecord.setOldValueSize(0);
+            }
+        }
+        logRecord.setUpdateLogSize();
+        txnSubsystem.getLogManager().log(logRecord);
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
index 0024717..44dcad8 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallback.java
@@ -16,12 +16,9 @@
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
@@ -33,22 +30,13 @@
  * Assumes LSM-BTrees as primary indexes.
  * Performs locking on primary keys, and also logs before/after images.
  */
-public class PrimaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+public class PrimaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback implements
         IModificationOperationCallback {
 
-    protected final long resourceId;
-    protected final byte resourceType;
-    protected final IndexOperation indexOp;
-    protected final ITransactionSubsystem txnSubsystem;
-
     public PrimaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields, ITransactionContext txnCtx,
             ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId, byte resourceType,
             IndexOperation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager);
-        this.resourceId = resourceId;
-        this.resourceType = resourceType;
-        this.indexOp = indexOp;
-        this.txnSubsystem = txnSubsystem;
+        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
     }
 
     @Override
@@ -64,7 +52,6 @@
     @Override
     public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
         try {
-            ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
             int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
             LSMBTreeTupleReference lsmBTreeTuple = (LSMBTreeTupleReference) before;
             IndexOperation oldOp = IndexOperation.INSERT;
@@ -74,8 +61,7 @@
             if (lsmBTreeTuple != null && lsmBTreeTuple.isAntimatter()) {
                 oldOp = IndexOperation.DELETE;
             }
-            ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
-                    indexOp, after, oldOp, before);
+            log(pkHash, after, oldOp, before);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
index 428a04e..abeec62 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/PrimaryIndexModificationOperationCallbackFactory.java
@@ -62,7 +62,7 @@
             IModificationOperationCallback modCallback = new PrimaryIndexModificationOperationCallback(datasetId,
                     primaryKeyFields, txnCtx, txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType,
                     indexOp);
-            txnCtx.registerIndexAndCallback(index, (AbstractOperationCallback) modCallback);
+            txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, true);
             return modCallback;
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
index e92f82c..ef5ad21 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallback.java
@@ -16,13 +16,10 @@
 package edu.uci.ics.asterix.transaction.management.opcallbacks;
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLogger;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
 import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
@@ -34,24 +31,16 @@
  * We assume that the modification of the corresponding primary index has already taken an appropriate lock.
  * This callback performs logging of the before and/or after images for secondary indexes.
  */
-public class SecondaryIndexModificationOperationCallback extends AbstractOperationCallback implements
+public class SecondaryIndexModificationOperationCallback extends AbstractIndexModificationOperationCallback implements
         IModificationOperationCallback {
 
-    protected final long resourceId;
-    protected final byte resourceType;
-    protected final IndexOperation indexOp;
     protected final IndexOperation oldOp;
-    protected final ITransactionSubsystem txnSubsystem;
 
     public SecondaryIndexModificationOperationCallback(int datasetId, int[] primaryKeyFields,
             ITransactionContext txnCtx, ILockManager lockManager, ITransactionSubsystem txnSubsystem, long resourceId,
             byte resourceType, IndexOperation indexOp) {
-        super(datasetId, primaryKeyFields, txnCtx, lockManager);
-        this.resourceId = resourceId;
-        this.resourceType = resourceType;
-        this.indexOp = indexOp;
+        super(datasetId, primaryKeyFields, txnCtx, lockManager, txnSubsystem, resourceId, resourceType, indexOp);
         oldOp = (indexOp == IndexOperation.DELETE) ? IndexOperation.INSERT : IndexOperation.DELETE;
-        this.txnSubsystem = txnSubsystem;
     }
 
     @Override
@@ -61,9 +50,7 @@
 
     @Override
     public void found(ITupleReference before, ITupleReference after) throws HyracksDataException {
-
         try {
-            ILogger logger = txnSubsystem.getTreeLoggerRepository().getIndexLogger(resourceId, resourceType);
             int pkHash = computePrimaryKeyHashValue(after, primaryKeyFields);
             IndexOperation effectiveOldOp;
             if (resourceType == ResourceType.LSM_BTREE) {
@@ -78,8 +65,7 @@
             } else {
                 effectiveOldOp = oldOp;
             }
-            ((IndexLogger) logger).generateLogRecord(txnSubsystem, txnCtx, datasetId.getId(), pkHash, resourceId,
-                    indexOp, after, effectiveOldOp, before);
+            this.log(pkHash, after, effectiveOldOp, before);
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
index 55ca399..d5bc877 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/opcallbacks/SecondaryIndexModificationOperationCallbackFactory.java
@@ -17,6 +17,7 @@
 
 import edu.uci.ics.asterix.common.context.ITransactionSubsystemProvider;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallbackFactory;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
@@ -55,8 +56,10 @@
 
         try {
             ITransactionContext txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(jobId);
-            return new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
+            IModificationOperationCallback modCallback = new SecondaryIndexModificationOperationCallback(datasetId, primaryKeyFields, txnCtx,
                     txnSubsystem.getLockManager(), txnSubsystem, resourceId, resourceType, indexOp);
+            txnCtx.registerIndexAndCallback(resourceId, index, (AbstractOperationCallback) modCallback, false);
+            return modCallback;
         } catch (ACIDException e) {
             throw new HyracksDataException(e);
         }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
index b09cd52..fdfc822 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/locking/LockManager.java
@@ -29,11 +29,13 @@
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.DatasetId;
 import edu.uci.ics.asterix.common.transactions.ILockManager;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
 import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogPage;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogPageReader;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants.LockManagerConstants.LockMode;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
@@ -78,15 +80,13 @@
     private DeadlockDetector deadlockDetector;
     private TimeOutDetector toutDetector;
     private DatasetId tempDatasetIdObj; //temporary object to avoid object creation
+    private JobId tempJobIdObj;
 
     private int tryLockDatasetGranuleRevertOperation;
 
     private LockRequestTracker lockRequestTracker; //for debugging
     private ConsecutiveWakeupContext consecutiveWakeupContext;
-
-    //TODO 
-    //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds. 
-    private LogicalLogLocator logicalLogLocator;
+    private final ILogRecord logRecord;
 
     public LockManager(TransactionSubsystem txnSubsystem) throws ACIDException {
         this.txnSubsystem = txnSubsystem;
@@ -94,22 +94,22 @@
         this.waiterLatch = new ReentrantReadWriteLock(true);
         this.jobHT = new HashMap<JobId, JobInfo>();
         this.datasetResourceHT = new HashMap<DatasetId, DatasetLockInfo>();
-        this.entityInfoManager = new EntityInfoManager(txnSubsystem.getTransactionProperties().getLockManagerShrinkTimer());
+        this.entityInfoManager = new EntityInfoManager(txnSubsystem.getTransactionProperties()
+                .getLockManagerShrinkTimer());
         this.lockWaiterManager = new LockWaiterManager();
         this.entityLockInfoManager = new EntityLockInfoManager(entityInfoManager, lockWaiterManager);
         this.deadlockDetector = new DeadlockDetector(jobHT, datasetResourceHT, entityLockInfoManager,
                 entityInfoManager, lockWaiterManager);
         this.toutDetector = new TimeOutDetector(this);
         this.tempDatasetIdObj = new DatasetId(0);
+        this.tempJobIdObj = new JobId(0);
         this.consecutiveWakeupContext = new ConsecutiveWakeupContext();
-
-        this.logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
-
+        this.logRecord = new LogRecord();
         if (IS_DEBUG_MODE) {
             this.lockRequestTracker = new LockRequestTracker();
         }
     }
-     
+
     public AsterixTransactionProperties getTransactionProperties() {
         return this.txnSubsystem.getTransactionProperties();
     }
@@ -200,7 +200,8 @@
                         if (doEscalate) {
                             throw new IllegalStateException(
                                     "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
-                                            + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+                                            + txnSubsystem.getTransactionProperties()
+                                                    .getEntityToDatasetLockEscalationThreshold());
                         }
                     }
                 }
@@ -640,15 +641,14 @@
     }
 
     @Override
-    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
-            throws ACIDException {
-        return internalUnlock(datasetId, entityHashValue, txnContext, false, false);
+    public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext) throws ACIDException {
+        internalUnlock(datasetId, entityHashValue, txnContext, false, false);
     }
 
     @Override
-    public boolean unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
+    public void unlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext, boolean commitFlag)
             throws ACIDException {
-        return internalUnlock(datasetId, entityHashValue, txnContext, false, commitFlag);
+        internalUnlock(datasetId, entityHashValue, txnContext, false, commitFlag);
     }
 
     private void instantUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext)
@@ -656,7 +656,7 @@
         internalUnlock(datasetId, entityHashValue, txnContext, true, false);
     }
 
-    private boolean internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
+    private void internalUnlock(DatasetId datasetId, int entityHashValue, ITransactionContext txnContext,
             boolean isInstant, boolean commitFlag) throws ACIDException {
         JobId jobId = txnContext.getJobId();
         int eLockInfo = -1;
@@ -665,7 +665,6 @@
         int entityInfo = -1;
         byte datasetLockMode;
 
-        boolean lockCountIsZero = false;
         if (IS_DEBUG_MODE) {
             if (entityHashValue == -1) {
                 throw new UnsupportedOperationException(
@@ -702,6 +701,22 @@
                         + "," + entityHashValue + "]: Corresponding lock info doesn't exist.");
             }
 
+            //////////////////////////////////////////////////////////
+            //[Notice]
+            //If both EntityLockCount and DatasetLockCount are 1, 
+            //then write entity-commit log and return without releasing the lock.
+            //The lock will be released when the entity-commit log is flushed. 
+            if (commitFlag && entityInfoManager.getEntityLockCount(entityInfo) == 1
+                    && entityInfoManager.getDatasetLockCount(entityInfo) == 1) {
+                if (txnContext.getTransactionType().equals(ITransactionContext.TransactionType.READ_WRITE)) {
+                    logRecord.formCommitLogRecord(txnContext, LogType.ENTITY_COMMIT, jobId.getId(), datasetId.getId(),
+                            entityHashValue);
+                    txnSubsystem.getLogManager().log(logRecord);
+                }
+                return;
+            }
+            //////////////////////////////////////////////////////////
+
             datasetLockMode = entityInfoManager.getDatasetLockMode(entityInfo) == LockMode.S ? LockMode.IS
                     : LockMode.IX;
 
@@ -713,28 +728,11 @@
 
             if (entityInfoManager.getEntityLockCount(entityInfo) == 0
                     && entityInfoManager.getDatasetLockCount(entityInfo) == 0) {
-                lockCountIsZero = true;
                 int threadCount = 0; //number of threads(in the same job) waiting on the same resource 
                 int waiterObjId = jobInfo.getFirstWaitingResource();
                 int waitingEntityInfo;
                 LockWaiter waiterObj;
 
-                //TODO
-                //This code should be taken care properly when there is a way to avoid doubling memory space for txnIds.
-                //This commit log is written here in order to avoid increasing the memory space for managing transactionIds
-                if (commitFlag) {
-                    if (txnContext.getTransactionType().equals(ITransactionContext.TransactionType.READ_WRITE)) {
-                        try {
-                            txnSubsystem.getLogManager().log(LogType.ENTITY_COMMIT, txnContext, datasetId.getId(),
-                                    entityHashValue, -1, (byte) 0, 0, null, null, logicalLogLocator);
-                        } catch (ACIDException e) {
-                            requestAbort(txnContext);
-                        }
-                    }
-
-                    txnContext.updateLastLSNForIndexes(logicalLogLocator.getLsn());
-                }
-
                 //1) wake up waiters and remove holder
                 //wake up waiters of dataset-granule lock
                 wakeUpDatasetLockWaiters(dLockInfo);
@@ -782,7 +780,8 @@
 
             if (ALLOW_ESCALATE_FROM_ENTITY_TO_DATASET) {
                 if (!isInstant && datasetLockMode == LockMode.IS) {
-                    jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+                    jobInfo.decreaseDatasetISLockCount(datasetId.getId(), txnSubsystem.getTransactionProperties()
+                            .getEntityToDatasetLockEscalationThreshold());
                 }
             }
 
@@ -793,7 +792,6 @@
         } finally {
             unlatchLockTable();
         }
-        return lockCountIsZero;
     }
 
     @Override
@@ -1297,7 +1295,8 @@
                             //We don't want to allow the lock escalation when there is a first lock request on a dataset. 
                             throw new IllegalStateException(
                                     "ESCALATE_TRHESHOLD_ENTITY_TO_DATASET should not be set to "
-                                            + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+                                            + txnSubsystem.getTransactionProperties()
+                                                    .getEntityToDatasetLockEscalationThreshold());
                         }
                     }
                 }
@@ -2097,7 +2096,8 @@
         try {
             StringBuilder sb = new StringBuilder();
             sb.append("\n>>dump_begin\t>>----- [ConfVars] -----");
-            sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: " + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
+            sb.append("\nESCALATE_TRHESHOLD_ENTITY_TO_DATASET: "
+                    + txnSubsystem.getTransactionProperties().getEntityToDatasetLockEscalationThreshold());
             sb.append("\nSHRINK_TIMER_THRESHOLD (entityLockInfoManager): "
                     + entityLockInfoManager.getShrinkTimerThreshold());
             sb.append("\nSHRINK_TIMER_THRESHOLD (entityInfoManager): " + entityInfoManager.getShrinkTimerThreshold());
@@ -2233,6 +2233,37 @@
             }
         }
     }
+
+    public void batchUnlock(LogPage logPage, LogPageReader logPageReader) throws ACIDException {
+        latchLockTable();
+        try {
+            ITransactionContext txnCtx = null;
+            LogRecord logRecord = logPageReader.next();
+            while (logRecord != null) {
+                if (logRecord.getLogType() == LogType.ENTITY_COMMIT) {
+                    tempDatasetIdObj.setId(logRecord.getDatasetId());
+                    tempJobIdObj.setId(logRecord.getJobId());
+                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+                    if (txnCtx == null) {
+                        throw new IllegalStateException("TransactionContext[" + tempJobIdObj + "] doesn't exist.");
+                    }
+                    unlock(tempDatasetIdObj, logRecord.getPKHashValue(), txnCtx);
+                    txnCtx.notifyOptracker(false);
+                } else if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+                    ((LogPage) logPage).notifyJobCommitter();
+                    tempJobIdObj.setId(logRecord.getJobId());
+                    txnCtx = txnSubsystem.getTransactionManager().getTransactionContext(tempJobIdObj);
+                    if (txnCtx == null) {
+                        throw new IllegalStateException("TransactionContext[" + tempJobIdObj + "] doesn't exist.");
+                    }
+                    txnCtx.notifyOptracker(true);
+                }
+                logRecord = logPageReader.next();
+            }
+        } finally {
+            unlatchLockTable();
+        }
+    }
 }
 
 class ConsecutiveWakeupContext {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
deleted file mode 100644
index 272ef5e..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/Buffer.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * Represents a wrapper over @see ByteBuffer supporting some custom APIs for
- * transaction support. This class is not "thread-safe". For performance
- * concerns, it is required for multiple writers to be able to write to the
- * buffer concurrently and that a writer is never blocked by another writer. The
- * users of this class must ensure that two concurrent writers get to write in
- * exclusive areas in the buffer. A reader and writer may or may not conflict
- * with each other. For example, reading of logs during roll back of a
- * transaction t1 does not conflict with writing of logs by another transaction
- * t2 as they are concerned with exclusive areas of the buffer. On the contrary,
- * a flushing the buffer to disk conflicts with a reader reading the buffer.
- * Appropriate locks are taken on the Buffer in the application logic and not
- * directly imposed by synchronized methods.
- */
-
-public class Buffer implements IBuffer {
-
-    ByteBuffer buffer;
-
-    public Buffer(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    protected Buffer() {
-    }
-
-    public void setBuffer(ByteBuffer buffer) {
-        this.buffer = buffer;
-    }
-
-    @Override
-    public byte getByte(int offset) {
-        return buffer.get(offset);
-    }
-
-    @Override
-    public byte getByte() {
-        return buffer.get();
-    }
-
-    @Override
-    public void getBytes(byte[] bytes, int offset, int size) {
-        System.arraycopy(buffer.array(), offset, bytes, 0, size);
-    }
-
-    @Override
-    public int getSize() {
-        return buffer.capacity();
-    }
-
-    @Override
-    public int readInt() {
-        return buffer.getInt();
-    }
-
-    @Override
-    public int readInt(int offset) {
-        return buffer.getInt(offset);
-    }
-
-    @Override
-    public long readLong(int offset) {
-        return buffer.getLong(offset);
-    }
-
-    @Override
-    public void put(byte b) {
-        buffer.put(b);
-    }
-
-    @Override
-    public void put(int offset, byte b) {
-        buffer.put(offset, b);
-    }
-
-    @Override
-    public void put(byte[] bytes, int start, int length) {
-        buffer.put(bytes, start, length);
-
-    }
-
-    @Override
-    public void put(byte[] bytes) {
-        buffer.put(bytes);
-    }
-
-    @Override
-    public void writeInt(int value) {
-        buffer.putInt(value);
-    }
-
-    @Override
-    public void writeInt(int offset, int value) {
-        buffer.putInt(offset, value);
-
-    }
-
-    @Override
-    public void writeLong(long value) {
-        buffer.putLong(value);
-    }
-
-    @Override
-    public void writeLong(int offset, long value) {
-        buffer.putLong(offset, value);
-
-    }
-
-    @Override
-    public byte[] getArray() {
-        return buffer.array();
-    }
-
-    @Override
-    public void erase() {
-        Arrays.fill(buffer.array(), (byte) 0);
-    }
-
-    @Override
-    public ByteBuffer getByteBuffer() {
-        return buffer;
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
deleted file mode 100644
index 79c95c4..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/DataUtil.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import java.util.zip.CRC32;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-
-/**
- * A utility class for doing bit level operations such as forming checksum or
- * converting between Integer and byte array. Used extensively during writing
- * and reading of logs.
- */
-public class DataUtil {
-
-    public static long getChecksum(IBuffer buffer, int offset, int length) {
-        CRC32 checksumEngine = new CRC32();
-        byte[] bytes = new byte[1];
-        for (int i = 0; i < length; i++) {
-            bytes[0] = buffer.getByte(offset + i);
-            checksumEngine.update(bytes, 0, 1);
-        }
-        return checksumEngine.getValue();
-    }
-
-    public static int byteArrayToInt(byte[] bytes, int offset) {
-        return ((bytes[offset] & 0xff) << 24) + ((bytes[offset + 1] & 0xff) << 16) + ((bytes[offset + 2] & 0xff) << 8)
-                + ((bytes[offset + 3] & 0xff) << 0);
-    }
-
-    public static byte[] intToByteArray(int value) {
-        byte[] bytes = new byte[4];
-        bytes[0] = (byte) ((value >>> 24) & 0xFF);
-        bytes[1] = (byte) ((value >>> 16) & 0xFF);
-        bytes[2] = (byte) ((value >>> 8) & 0xFF);
-        bytes[3] = (byte) ((value >>> 0) & 0xFF);
-        return bytes;
-    }
-
-    public static long byteArrayToLong(byte[] bytes, int offset) {
-        return ((bytes[offset] & 0xff) << 56) + ((bytes[offset + 1] & 0xff) << 48) + ((bytes[offset + 2] & 0xff) << 40)
-                + ((bytes[offset + 3] & 0xff) << 32) + ((bytes[offset + 4] & 0xff) << 24)
-                + ((bytes[offset + 5] & 0xff) << 16) + ((bytes[offset + 6] & 0xff) << 8)
-                + ((bytes[offset + 7] & 0xff) << 0);
-    }
-
-    public static byte[] longToByteArray(long value) {
-        byte[] bytes = new byte[8];
-        bytes[0] = (byte) ((value >>> 56) & 0xFF);
-        bytes[1] = (byte) ((value >>> 48) & 0xFF);
-        bytes[2] = (byte) ((value >>> 40) & 0xFF);
-        bytes[3] = (byte) ((value >>> 32) & 0xFF);
-        bytes[4] = (byte) ((value >>> 24) & 0xFF);
-        bytes[5] = (byte) ((value >>> 16) & 0xFF);
-        bytes[6] = (byte) ((value >>> 8) & 0xFF);
-        bytes[7] = (byte) ((value >>> 0) & 0xFF);
-        return bytes;
-    }
-
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
deleted file mode 100644
index 0de43cc..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/GenericFilter.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-
-/*
- *  A generic filter that encompasses one or more filters (see @ILogFiler) that may be applied when selectively retrieving logs.
- *  The contained filters are assumed to form a conjunction.   
- */
-public class GenericFilter implements ILogFilter {
-
-    private final List<ILogFilter> logFilters;
-
-    public GenericFilter() {
-        logFilters = new ArrayList<ILogFilter>();
-    }
-
-    public GenericFilter(List<ILogFilter> logFilters) {
-        this.logFilters = logFilters;
-    }
-
-    public boolean accept(IBuffer fileBuffer, long offset, int length) {
-        boolean satisfies = true;
-        for (ILogFilter logFilter : logFilters) {
-            satisfies = satisfies && logFilter.accept(fileBuffer, offset, length);
-            if (!satisfies) {
-                break;
-            }
-        }
-        return satisfies;
-    }
-
-    public void addFilter(ILogFilter logFilter) {
-        logFilters.add(logFilter);
-    }
-
-    public boolean removeFilter(ILogFilter logFilter) {
-        return logFilters.remove(logFilter);
-    }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
deleted file mode 100644
index 755e85b..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLogger.java
+++ /dev/null
@@ -1,214 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ICloseable;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
-
-public class IndexLogger implements ILogger, ICloseable {
-
-    private final Map<Object, Object> jobId2ReusableLogContentObjectRepositoryMap = new ConcurrentHashMap<Object, Object>();
-
-    public static final String TREE_INDEX = "TREE_INDEX";
-    public static final String TUPLE_REFERENCE = "TUPLE_REFERENCE";
-    public static final String TUPLE_WRITER = "TUPLE_WRITER";
-    public static final String INDEX_OPERATION = "INDEX_OPERATION";
-    public static final String RESOURCE_ID = "RESOURCE_ID";
-
-    private final long resourceId;
-    private final byte resourceType;
-    private final SimpleTupleWriter tupleWriter;
-
-    public class BTreeOperationCodes {
-        public static final byte INSERT = 0;
-        public static final byte DELETE = 1;
-    }
-
-    public IndexLogger(long resourceId, byte resourceType, IIndex index) {
-        this.resourceId = resourceId;
-        this.resourceType = resourceType;
-        this.tupleWriter = new SimpleTupleWriter();
-    }
-
-    public synchronized void close(ITransactionContext context) {
-        ReusableLogContentObjectRepository txnThreadStateRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
-                .get(context.getJobId());
-        txnThreadStateRepository.remove(Thread.currentThread().getId());
-        jobId2ReusableLogContentObjectRepositoryMap.remove(context.getJobId());
-    }
-
-    public void generateLogRecord(ITransactionSubsystem txnSubsystem, ITransactionContext context, int datasetId,
-            int PKHashValue, long resourceId, IndexOperation newOperation, ITupleReference newValue,
-            IndexOperation oldOperation, ITupleReference oldValue) throws ACIDException {
-
-        if (this.resourceId != resourceId) {
-            throw new ACIDException("IndexLogger mistach");
-        }
-
-        context.addCloseableResource(this); // the close method would be called
-        // on this TreeLogger instance at
-        // the time of transaction
-        // commit/abort.
-        if (newOperation != IndexOperation.INSERT && newOperation != IndexOperation.DELETE) {
-            throw new ACIDException("Loging for Operation " + newOperation + " not supported");
-        }
-
-        ReusableLogContentObject reusableLogContentObject = null;
-        ReusableLogContentObjectRepository reusableLogContentObjectRepository = null;
-        reusableLogContentObjectRepository = (ReusableLogContentObjectRepository) jobId2ReusableLogContentObjectRepositoryMap
-                .get(context.getJobId());
-        if (reusableLogContentObjectRepository == null) {
-            synchronized (context) { // threads belonging to different
-                // transaction do not need to
-                // synchronize amongst them.
-                if (reusableLogContentObjectRepository == null) {
-                    reusableLogContentObjectRepository = new ReusableLogContentObjectRepository();
-                    jobId2ReusableLogContentObjectRepositoryMap.put(context.getJobId(),
-                            reusableLogContentObjectRepository);
-                }
-            }
-        }
-
-        reusableLogContentObject = reusableLogContentObjectRepository.getObject(Thread.currentThread().getId());
-        if (reusableLogContentObject == null) {
-            LogicalLogLocator logicalLogLocator = LogUtil.getDummyLogicalLogLocator(txnSubsystem.getLogManager());
-            reusableLogContentObject = new ReusableLogContentObject(logicalLogLocator, newOperation, newValue,
-                    oldOperation, oldValue);
-            reusableLogContentObjectRepository.putObject(Thread.currentThread().getId(), reusableLogContentObject);
-        } else {
-            reusableLogContentObject.setNewOperation(newOperation);
-            reusableLogContentObject.setNewValue(newValue);
-            reusableLogContentObject.setOldOperation(oldOperation);
-            reusableLogContentObject.setOldValue(oldValue);
-        }
-
-        int logContentSize = 4/*TupleFieldCount*/+ 1/*NewOperation*/+ 4/*newValueLength*/;
-        if (newValue != null) {
-            logContentSize += tupleWriter.bytesRequired(newValue);
-        }
-
-        logContentSize += 1/*OldOperation*/+ 4/*oldValueLength*/;
-        if (oldValue != null) {
-            logContentSize += tupleWriter.bytesRequired(oldValue);
-        }
-
-        txnSubsystem.getLogManager().log(LogType.UPDATE, context, datasetId, PKHashValue, resourceId, resourceType,
-                logContentSize, reusableLogContentObject, this, reusableLogContentObject.getLogicalLogLocator());
-    }
-
-    @Override
-    public void log(ITransactionContext context, LogicalLogLocator logicalLogLocator, int logContentSize,
-            ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        int offset = 0;
-        int tupleSize = 0;
-
-        //tuple field count
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, reusableLogContentObject
-                .getNewValue().getFieldCount());
-        offset += 4;
-
-        //new operation
-        (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + offset,
-                (byte) reusableLogContentObject.getNewOperation().ordinal());
-        offset += 1;
-
-        //new tuple size
-        if (reusableLogContentObject.getNewValue() != null) {
-            tupleSize = tupleWriter.bytesRequired(reusableLogContentObject.getNewValue());
-        }
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, tupleSize);
-        offset += 4;
-
-        //new tuple
-        if (tupleSize != 0) {
-            tupleWriter.writeTuple(reusableLogContentObject.getNewValue(), logicalLogLocator.getBuffer().getArray(),
-                    logicalLogLocator.getMemoryOffset() + offset);
-            offset += tupleSize;
-        }
-
-        if (resourceType == ResourceType.LSM_BTREE) {
-            //old operation
-            (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + offset,
-                    (byte) reusableLogContentObject.getOldOperation().ordinal());
-            offset += 1;
-
-            if (reusableLogContentObject.getOldOperation() != IndexOperation.NOOP) {
-                //old tuple size
-                if (reusableLogContentObject.getOldValue() != null) {
-                    tupleSize = tupleWriter.bytesRequired(reusableLogContentObject.getOldValue());
-                } else {
-                    tupleSize = 0;
-                }
-                (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + offset, tupleSize);
-                offset += 4;
-
-                if (tupleSize != 0) {
-                    //old tuple
-                    tupleWriter.writeTuple(reusableLogContentObject.getOldValue(), logicalLogLocator.getBuffer()
-                            .getArray(), logicalLogLocator.getMemoryOffset() + offset);
-                }
-            }
-        }
-    }
-
-    @Override
-    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
-            throws ACIDException {
-    }
-
-    @Override
-    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject)
-            throws ACIDException {
-    }
-
-    /**
-     * Represents a utility class for generating log records corresponding to
-     * operations on a ITreeIndex implementation. A TreeLogger instance is thread
-     * safe and can be shared across multiple threads that may belong to same or
-     * different transactions.
-     */
-    public class ReusableLogContentObjectRepository {
-
-        private final Map<Long, ReusableLogContentObject> id2Object = new HashMap<Long, ReusableLogContentObject>();
-
-        public synchronized ReusableLogContentObject getObject(long threadId) {
-            return id2Object.get(threadId);
-        }
-
-        public synchronized void putObject(long threadId, ReusableLogContentObject reusableLogContentObject) {
-            this.id2Object.put(threadId, reusableLogContentObject);
-        }
-
-        public synchronized void remove(long threadId) {
-            id2Object.remove(threadId);
-        }
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
deleted file mode 100644
index b95b943..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexLoggerRepository.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ILoggerRepository;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.MutableResourceId;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-
-public class IndexLoggerRepository implements ILoggerRepository {
-
-    private final Map<MutableResourceId, ILogger> loggers = new HashMap<MutableResourceId, ILogger>();
-    private final ITransactionSubsystem txnSubsystem;
-    private MutableResourceId mutableResourceId;
-
-    public IndexLoggerRepository(ITransactionSubsystem provider) {
-        this.txnSubsystem = provider;
-        mutableResourceId = new MutableResourceId(0);
-    }
-
-    @Override
-    public synchronized ILogger getIndexLogger(long resourceId, byte resourceType) throws ACIDException {
-        mutableResourceId.setId(resourceId);
-        ILogger logger = loggers.get(mutableResourceId);
-        if (logger == null) {
-            MutableResourceId newMutableResourceId = new MutableResourceId(resourceId);
-            IIndex index;
-            try {
-                index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                        .getIndex(resourceId);
-            } catch (HyracksDataException e) {
-                throw new ACIDException(e);
-            }
-            logger = new IndexLogger(resourceId, resourceType, index);
-            loggers.put(newMutableResourceId, logger);
-        }
-        return logger;
-    }
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
deleted file mode 100644
index 16ffa69..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/IndexResourceManager.java
+++ /dev/null
@@ -1,162 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
-import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
-import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleReference;
-import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
-
-public class IndexResourceManager implements IResourceManager {
-
-    public final byte resourceType;
-
-    private final ITransactionSubsystem txnSubsystem;
-
-    public IndexResourceManager(byte resourceType, ITransactionSubsystem provider) {
-        this.resourceType = resourceType;
-        this.txnSubsystem = provider;
-    }
-
-    public byte getResourceManagerId() {
-        return resourceType;
-    }
-
-    public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
-        long resourceId = logRecordHelper.getResourceId(logLocator);
-        int offset = logRecordHelper.getLogContentBeginPos(logLocator);
-
-        IIndex index;
-        try {
-            index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(resourceId);
-        } catch (HyracksDataException e1) {
-            throw new ACIDException("Cannot undo: unable to find index");
-        }
-
-        /* field count */
-        int fieldCount = logLocator.getBuffer().readInt(offset);
-        offset += 4;
-
-        /* new operation */
-        byte newOperation = logLocator.getBuffer().getByte(offset);
-        offset += 1;
-
-        /* new value size */
-        int newValueSize = logLocator.getBuffer().readInt(offset);
-        offset += 4;
-
-        /* new value */
-        SimpleTupleWriter tupleWriter = new SimpleTupleWriter();
-        SimpleTupleReference newTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
-        newTuple.setFieldCount(fieldCount);
-        newTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
-        offset += newValueSize;
-
-        try {
-            ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
-            if (resourceType == ResourceType.LSM_BTREE) {
-
-                /* old operation */
-                byte oldOperation = logLocator.getBuffer().getByte(offset);
-                offset += 1;
-
-                if (oldOperation != (byte) IndexOperation.NOOP.ordinal()) {
-                    /* old value size */
-                    int oldValueSize = logLocator.getBuffer().readInt(offset);
-                    offset += 4;
-
-                    /* old value */
-                    SimpleTupleReference oldTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
-                    oldTuple.setFieldCount(fieldCount);
-                    oldTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
-                    offset += oldValueSize;
-
-                    if (oldOperation == (byte) IndexOperation.DELETE.ordinal()) {
-                        indexAccessor.forceDelete(oldTuple);
-                    } else {
-                        indexAccessor.forceInsert(oldTuple);
-                    }
-                } else {
-                    indexAccessor.forcePhysicalDelete(newTuple);
-                }
-            } else {
-                if (newOperation == (byte) IndexOperation.DELETE.ordinal()) {
-                    indexAccessor.forceInsert(newTuple);
-                } else {
-                    indexAccessor.forceDelete(newTuple);
-                }
-            }
-        } catch (Exception e) {
-            throw new ACIDException("Undo failed", e);
-        }
-    }
-
-    public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator logLocator) throws ACIDException {
-        long resourceId = logRecordHelper.getResourceId(logLocator);
-        int offset = logRecordHelper.getLogContentBeginPos(logLocator);
-
-        IIndex index;
-        try {
-            index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
-                    .getIndex(resourceId);
-        } catch (HyracksDataException e1) {
-            throw new ACIDException("Cannot redo: unable to find index");
-        }
-
-        /* field count */
-        int fieldCount = logLocator.getBuffer().readInt(offset);
-        offset += 4;
-
-        /* new operation */
-        byte newOperation = logLocator.getBuffer().getByte(offset);
-        offset += 1;
-
-        /* new value size */
-        int newValueSize = logLocator.getBuffer().readInt(offset);
-        offset += 4;
-
-        /* new value */
-        SimpleTupleWriter tupleWriter = new SimpleTupleWriter();
-        SimpleTupleReference newTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
-        newTuple.setFieldCount(fieldCount);
-        newTuple.resetByTupleOffset(logLocator.getBuffer().getByteBuffer(), offset);
-        offset += newValueSize;
-
-        try {
-            ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE, NoOpOperationCallback.INSTANCE);
-
-            if (newOperation == IndexOperation.INSERT.ordinal()) {
-                indexAccessor.insert(newTuple);
-            } else if (newOperation == IndexOperation.DELETE.ordinal()) {
-                indexAccessor.delete(newTuple);
-            } else {
-                new ACIDException("Unsupported operation type for undo operation : " + newOperation);
-            }
-        } catch (Exception e) {
-            throw new ACIDException("Redo failed", e);
-        }
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java
deleted file mode 100644
index 5f558f3..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogActionType.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-public class LogActionType {
-
-    public static final byte REDO = 0; // used for a log record that contains
-    // just redo information.
-    public static final byte REDO_UNDO = 1; // used for a log record that
-    // contains both redo and undo
-    // information.
-    public static final byte UNDO = 2; // used for a log record that contains
-    // just undo information.
-    public static final byte NO_OP = 3; // used for a log record that does not
-    // require any action.
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
deleted file mode 100644
index d3e1ad1..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogCursor.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import java.io.File;
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-public class LogCursor implements ILogCursor {
-
-    private final LogManager logManager;
-    private final ILogFilter logFilter;
-    private final int logPageSize;
-    private IBuffer readOnlyBuffer;
-    private LogicalLogLocator logicalLogLocator = null;
-    private boolean needReloadBuffer = true;
-
-    public LogCursor(final LogManager logManager, PhysicalLogLocator startingPhysicalLogLocator, ILogFilter logFilter,
-            int logPageSize) throws IOException, ACIDException {
-        this.logFilter = logFilter;
-        this.logManager = logManager;
-        this.logPageSize = logPageSize;
-        initialize(startingPhysicalLogLocator);
-    }
-
-    private void initialize(final PhysicalLogLocator startingPhysicalLogLocator) throws IOException, ACIDException {
-        logicalLogLocator = new LogicalLogLocator(startingPhysicalLogLocator.getLsn(), null, 0, logManager);
-    }
-
-    private IFileBasedBuffer getReadOnlyBuffer(long lsn, int size) throws IOException {
-        int fileId = (int) (lsn / logManager.getLogManagerProperties().getLogPartitionSize());
-        String filePath = LogUtil.getLogFilePath(logManager.getLogManagerProperties(), fileId);
-        File file = new File(filePath);
-        if (file.exists()) {
-            return FileUtil.getFileBasedBuffer(filePath, lsn
-                    % logManager.getLogManagerProperties().getLogPartitionSize(), size, logManager
-                    .getLogManagerProperties().getDiskSectorSize());
-        } else {
-            return null;
-        }
-    }
-
-    /**
-     * Moves the cursor to the next log record that satisfies the configured
-     * filter. The parameter nextLogLocator is set to the point to the next log
-     * record.
-     * 
-     * @param currentLogLocator
-     * @return true if the cursor was successfully moved to the next log record
-     *         false if there are no more log records that satisfy the
-     *         configured filter.
-     */
-    @Override
-    public boolean next(LogicalLogLocator currentLogLocator) throws IOException, ACIDException {
-
-        //TODO
-        //Test the correctness when multiple log files are created
-        int integerRead = -1;
-        boolean logRecordBeginPosFound = false;
-        long bytesSkipped = 0;
-
-        //if the lsn to read is greater than or equal to the most recent lsn, then return false
-        if (logicalLogLocator.getLsn() >= logManager.getCurrentLsn().get()) {
-            return false;
-        }
-
-        if (logManager.isMemoryRead(logicalLogLocator.getLsn())) {
-            return readFromMemory(currentLogLocator);
-        }
-
-        //if the readOnlyBuffer should be reloaded, then load the log page from the log file.
-        //needReloadBuffer is set to true if the log record is read from the memory log page.
-        if (needReloadBuffer) {
-            //log page size doesn't exceed integer boundary
-            int offset = (int) (logicalLogLocator.getLsn() % logPageSize);
-            long adjustedLSN = logicalLogLocator.getLsn() - offset;
-            readOnlyBuffer = getReadOnlyBuffer(adjustedLSN, logPageSize);
-            logicalLogLocator.setBuffer(readOnlyBuffer);
-            logicalLogLocator.setMemoryOffset(offset);
-            needReloadBuffer = false;
-        }
-
-        //check whether the currentOffset has enough space to have new log record by comparing
-        //the smallest log record type(which is commit)'s log header.
-        while (logicalLogLocator.getMemoryOffset() <= readOnlyBuffer.getSize()
-                - logManager.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
-            integerRead = readOnlyBuffer.readInt(logicalLogLocator.getMemoryOffset());
-            if (integerRead == LogManagerProperties.LOG_MAGIC_NUMBER) {
-                logRecordBeginPosFound = true;
-                break;
-            }
-            logicalLogLocator.increaseMemoryOffset(1);
-            logicalLogLocator.incrementLsn();
-            bytesSkipped++;
-            if (bytesSkipped > logPageSize) {
-                return false; // the maximum size of a log record is limited to
-                // a log page size. If we have skipped as many
-                // bytes without finding a log record, it
-                // indicates an absence of logs any further.
-            }
-
-            if (logManager.isMemoryRead(logicalLogLocator.getLsn())) {
-                return next(currentLogLocator); //should read from memory if there is any further log
-            }
-        }
-
-        if (!logRecordBeginPosFound) {
-            // need to reload the buffer
-            // TODO
-            // reduce IO by reading more pages(equal to logBufferSize) at a time.
-            long lsnpos = ((logicalLogLocator.getLsn() / logPageSize) + 1) * logPageSize;
-
-            readOnlyBuffer = getReadOnlyBuffer(lsnpos, logPageSize);
-            if (readOnlyBuffer != null) {
-                logicalLogLocator.setBuffer(readOnlyBuffer);
-                logicalLogLocator.setLsn(lsnpos);
-                logicalLogLocator.setMemoryOffset(0);
-                return next(currentLogLocator);
-            } else {
-                return false;
-            }
-        }
-
-        int logLength = logManager.getLogRecordHelper().getLogRecordSize(
-                logManager.getLogRecordHelper().getLogType(logicalLogLocator),
-                logManager.getLogRecordHelper().getLogContentSize(logicalLogLocator));
-        if (logManager.getLogRecordHelper().validateLogRecord(logicalLogLocator)) {
-            if (currentLogLocator == null) {
-                currentLogLocator = new LogicalLogLocator(0, readOnlyBuffer, -1, logManager);
-            }
-            currentLogLocator.setLsn(logicalLogLocator.getLsn());
-            currentLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset());
-            currentLogLocator.setBuffer(readOnlyBuffer);
-            logicalLogLocator.incrementLsn(logLength);
-            logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
-        } else {
-            throw new ACIDException("Invalid Log Record found ! checksums do not match :( ");
-        }
-        return logFilter.accept(readOnlyBuffer, currentLogLocator.getMemoryOffset(), logLength);
-    }
-
-    /**
-     * Returns the filter associated with the cursor.
-     * 
-     * @return ILogFilter
-     */
-    @Override
-    public ILogFilter getLogFilter() {
-        return logFilter;
-    }
-
-    private boolean readFromMemory(LogicalLogLocator currentLogLocator) throws ACIDException, IOException {
-        byte[] logRecord = null;
-        long lsn = logicalLogLocator.getLsn();
-
-        //set the needReloadBuffer to true
-        needReloadBuffer = true;
-
-        int pageIndex = logManager.getLogPageIndex(lsn);
-        logicalLogLocator.setMemoryOffset(logManager.getLogPageOffset(lsn));
-
-        // take a lock on the log page so that the page is not flushed to
-        // disk interim
-        IFileBasedBuffer logPage = logManager.getLogPage(pageIndex);
-        synchronized (logPage) {
-            // need to check again if the log record in the log buffer or has reached the disk
-            if (logManager.isMemoryRead(lsn)) {
-
-                //find the magic number to identify the start of the log record
-                //----------------------------------------------------------------
-                int readNumber = -1;
-                int logMagicNumber = LogManagerProperties.LOG_MAGIC_NUMBER;
-                int bytesSkipped = 0;
-                boolean logRecordBeginPosFound = false;
-                //check whether the currentOffset has enough space to have new log record by comparing
-                //the smallest log record type(which is commit)'s log header.
-                while (logicalLogLocator.getMemoryOffset() <= logPageSize
-                        - logManager.getLogRecordHelper().getLogHeaderSize(LogType.COMMIT)) {
-                    readNumber = logPage.readInt(logicalLogLocator.getMemoryOffset());
-                    if (readNumber == logMagicNumber) {
-                        logRecordBeginPosFound = true;
-                        break;
-                    }
-                    logicalLogLocator.increaseMemoryOffset(1);
-                    logicalLogLocator.incrementLsn();
-                    bytesSkipped++;
-                    if (bytesSkipped > logPageSize) {
-                        return false; // the maximum size of a log record is limited to
-                        // a log page size. If we have skipped as many
-                        // bytes without finding a log record, it
-                        // indicates an absence of logs any further.
-                    }
-                }
-
-                if (!logRecordBeginPosFound) {
-                    // need to read the next log page
-                    readOnlyBuffer = null;
-                    logicalLogLocator.setBuffer(null);
-                    lsn = ((logicalLogLocator.getLsn() / logPageSize) + 1) * logPageSize;
-                    logicalLogLocator.setLsn(lsn);
-                    logicalLogLocator.setMemoryOffset(0);
-                    return next(currentLogLocator);
-                }
-                //------------------------------------------------------
-
-                logicalLogLocator.setBuffer(logPage);
-                int logLength = logManager.getLogRecordHelper().getLogRecordSize(
-                        logManager.getLogRecordHelper().getLogType(logicalLogLocator),
-                        logManager.getLogRecordHelper().getLogContentSize(logicalLogLocator));
-                logRecord = new byte[logLength];
-
-                //copy the log record and set the buffer of logical log locator to the buffer of the copied log record.
-                System.arraycopy(logPage.getArray(), logicalLogLocator.getMemoryOffset(), logRecord, 0, logLength);
-                MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
-                readOnlyBuffer = memBuffer;
-                logicalLogLocator.setBuffer(readOnlyBuffer);
-                logicalLogLocator.setMemoryOffset(0);
-
-                if (logManager.getLogRecordHelper().validateLogRecord(logicalLogLocator)) {
-                    if (currentLogLocator == null) {
-                        currentLogLocator = new LogicalLogLocator(0, readOnlyBuffer, -1, logManager);
-                    }
-                    currentLogLocator.setLsn(logicalLogLocator.getLsn());
-                    currentLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset());
-                    currentLogLocator.setBuffer(readOnlyBuffer);
-                    logicalLogLocator.incrementLsn(logLength);
-                    logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
-                } else {
-                    //if the checksum doesn't match, there is two possible scenario. 
-                    //case1) the log file corrupted: there's nothing we can do for this case during abort. 
-                    //case2) the log record is partially written by another thread. So, we may ignore this log record 
-                    //       and continue to read the next log record
-                    //[NOTICE]
-                    //Only case2 is handled here. 
-                    logicalLogLocator.incrementLsn(logLength);
-                    logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset() + logLength);
-                    return next(currentLogLocator);
-                }
-                return logFilter.accept(readOnlyBuffer, currentLogLocator.getMemoryOffset(), logLength);
-
-            } else {
-                return next(currentLogLocator);//read from disk
-            }
-        }
-    }
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
index 0bc9462..49c07c1 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogManager.java
@@ -15,731 +15,156 @@
 package edu.uci.ics.asterix.transaction.management.service.logging;
 
 import java.io.File;
+import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.OutputStream;
 import java.io.RandomAccessFile;
-import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
-import java.util.HashMap;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import edu.uci.ics.asterix.common.api.AsterixThreadExecutor;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.FileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.IFileBasedBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
 import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.ILogger;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
 
 public class LogManager implements ILogManager, ILifeCycleComponent {
 
     public static final boolean IS_DEBUG_MODE = false;// true
     private static final Logger LOGGER = Logger.getLogger(LogManager.class.getName());
-    private final TransactionSubsystem provider;
-    private LogManagerProperties logManagerProperties;
-    private LogPageFlushThread logPageFlusher;
+    private final TransactionSubsystem txnSubsystem;
+    private final LogManagerProperties logManagerProperties;
+    private final long logFileSize;
     private final int logPageSize;
-    private long statLogSize;
-    private long statLogCount;
+    private final int numLogPages;
+    private final String logDir;
+    private final String logFilePrefix;
+    private final MutableLong flushLSN;
+    private LinkedBlockingQueue<LogPage> emptyQ;
+    private LinkedBlockingQueue<LogPage> flushQ;
+    private long appendLSN;
+    private FileChannel appendChannel;
+    private LogPage appendPage;
+    private LogFlusher logFlusher;
 
-    /*
-     * the array of log pages. The number of log pages is configurable. Pages
-     * taken together form an in-memory log buffer.
-     */
-    private IFileBasedBuffer[] logPages;
-
-    private ILogRecordHelper logRecordHelper;
-
-    /*
-     * Number of log pages that constitute the in-memory log buffer.
-     */
-    private int numLogPages;
-
-    private AtomicLong lastFlushedLSN = new AtomicLong(-1);
-
-    /*
-     * When the transaction eco-system comes to life, the log manager positions
-     * itself to the end of the last written log. the startingLsn represent the
-     * lsn value of the next log record to be written after a system (re)start.
-     * The value is zero when the system is starting for the first time.
-     */
-    private long startingLSN = 0;
-
-    /*
-     * lsn represents the monotonically increasing long value that can be broken
-     * down into a file id and an offset within a log file.
-     */
-    private AtomicLong lsn = new AtomicLong(0);
-
-    private List<HashMap<ITransactionContext, Integer>> activeTxnCountMaps;
-
-    public void addFlushRequest(int pageIndex, long lsn, boolean isSynchronous) {
-        logPageFlusher.requestFlush(pageIndex, lsn, isSynchronous);
-    }
-
-    public AtomicLong getLastFlushedLsn() {
-        return lastFlushedLSN;
-    }
-
-    public AtomicLong getCurrentLsn() {
-        return lsn;
-    }
-
-    public long incrementLastFlushedLsn(long delta) {
-        return lastFlushedLSN.addAndGet(delta);
-    }
-
-    public LogManager(TransactionSubsystem provider) throws ACIDException {
-        this.provider = provider;
-        logManagerProperties = new LogManagerProperties(this.provider.getTransactionProperties(), this.provider.getId());
+    public LogManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+        this.txnSubsystem = txnSubsystem;
+        logManagerProperties = new LogManagerProperties(this.txnSubsystem.getTransactionProperties(),
+                this.txnSubsystem.getId());
+        logFileSize = logManagerProperties.getLogPartitionSize();
         logPageSize = logManagerProperties.getLogPageSize();
-        initLogManager();
-        statLogSize = 0;
-        statLogCount = 0;
-    }
-
-    public LogManager(TransactionSubsystem provider, String nodeId) throws ACIDException {
-        this.provider = provider;
-        logManagerProperties = new LogManagerProperties(provider.getTransactionProperties(), nodeId);
-        logPageSize = logManagerProperties.getLogPageSize();
-        initLogManager();
-        statLogSize = 0;
-        statLogCount = 0;
-    }
-
-    private void initLogManager() throws ACIDException {
-        logRecordHelper = new LogRecordHelper(this);
         numLogPages = logManagerProperties.getNumLogPages();
-        activeTxnCountMaps = new ArrayList<HashMap<ITransactionContext, Integer>>(numLogPages);
+        logDir = logManagerProperties.getLogDir();
+        logFilePrefix = logManagerProperties.getLogFilePrefix();
+        flushLSN = new MutableLong();
+        initializeLogManager();
+    }
 
+    private void initializeLogManager() {
+        emptyQ = new LinkedBlockingQueue<LogPage>(numLogPages);
+        flushQ = new LinkedBlockingQueue<LogPage>(numLogPages);
         for (int i = 0; i < numLogPages; i++) {
-            activeTxnCountMaps.add(new HashMap<ITransactionContext, Integer>());
+            emptyQ.offer(new LogPage((LockManager) txnSubsystem.getLockManager(), logPageSize, flushLSN));
         }
-
-        logPages = new FileBasedBuffer[numLogPages];
-
-        /*
-         * place the log anchor at the end of the last log record written.
-         */
-        initLSN();
-
-        /*
-         * initialize the log pages.
-         */
-        initializeLogPages(startingLSN);
-
-        /*
-         * Instantiate and begin the LogFlusher thread. The Log Flusher thread
-         * is responsible for putting log pages to disk. It is configured as a
-         * daemon thread so that it does not stop the JVM from exiting when all
-         * other threads are done with their work.
-         */
-        logPageFlusher = new LogPageFlushThread(this);
-        logPageFlusher.setDaemon(true);
-        AsterixThreadExecutor.INSTANCE.execute(logPageFlusher);
-    }
-
-    public int getLogPageIndex(long lsnValue) {
-        return (int) (((lsnValue - startingLSN) / logPageSize) % numLogPages);
-    }
-
-    /*
-     * given a lsn, get the file id where the corresponding log record is
-     * located.
-     */
-    public int getLogFileId(long lsnValue) {
-        return (int) ((lsnValue) / logManagerProperties.getLogPartitionSize());
-    }
-
-    /*
-     * given a lsn, get the offset within a log page where the corresponding log
-     * record is (to be) placed.
-     */
-    public int getLogPageOffset(long lsnValue) {
-        return (int) (lsnValue % logPageSize);
-    }
-
-    /*
-     * The method that reserves the space for a transaction to write log record
-     * in the log buffer. Note that the method is not synchronized for
-     * performance reasons as we do not want transactions to be blocked by each
-     * other when writing log records.
-     * 
-     * @param entrySize: the requested space.
-     * 
-     * @param logType: the type of log record.
-     */
-    private long getLsn(int entrySize, byte logType) throws ACIDException {
-
-        while (true) {
-            boolean forwardPage = false;
-            long old = lsn.get();
-
-            // get the log page corresponding to the current lsn value
-            int pageIndex = getLogPageIndex(old);
-            long retVal = old;
-
-            // the lsn value for the next request if the current request is
-            // served.
-            long next = old + entrySize;
-            int prevPage = -1;
-
-            // check if the log record will cross page boundaries, a case that
-            // is not allowed.
-            if ((next - 1) / logPageSize != old / logPageSize || (next % logPageSize == 0)) {
-
-                if ((old != 0 && old % logPageSize == 0)) {
-                    // On second thought, this shall never be the case as it
-                    // means that the lsn is
-                    // currently at the beginning of a page and we still need to
-                    // forward the page which
-                    // means that the entrySize exceeds a log page size. If this
-                    // is the case, an
-                    // exception is thrown before calling this API. would remove
-                    // this case.
-                    retVal = old;
-
-                } else {
-                    // set the lsn to point to the beginning of the next page.
-                    retVal = ((old / logPageSize) + 1) * logPageSize;
-                }
-
-                next = retVal;
-
-                // as the log record shall cross log page boundary, we must
-                // re-assign the lsn so
-                // that the log record begins on a different location.
-                forwardPage = true;
-
-                prevPage = pageIndex;
-                pageIndex = getNextPageInSequence(pageIndex);
-            }
-
-            if (!lsn.compareAndSet(old, next)) {
-                // Atomic call -> returns true only when the value represented
-                // by lsn is same as
-                // "old". The value is updated to "next".
-                continue;
-            }
-
-            if (forwardPage) {
-                logPages[prevPage].acquireReadLatch();
-                // increment the counter as the transaction thread now holds a
-                // space in the log page and hence is an owner.
-                logPages[prevPage].incRefCnt();
-                logPages[prevPage].releaseReadLatch();
-
-                // forward the nextWriteOffset in the log page
-                logPages[prevPage].setBufferNextWriteOffset(logPageSize);
-
-                logPages[prevPage].decRefCnt();
-
-                addFlushRequest(prevPage, old, false);
-
-                // The transaction thread that discovers the need to forward a
-                // page is made to re-acquire a lsn.
-                continue;
-
-            } else {
-                logPages[pageIndex].acquireReadLatch();
-                // increment the counter as the transaction thread now holds a
-                // space in the log page and hence is an owner.
-                logPages[pageIndex].incRefCnt();
-                logPages[pageIndex].releaseReadLatch();
-
-                // Before the count is incremented, if the flusher flushed the
-                // allocated page,
-                // then retry to get new LSN. Otherwise, the log with allocated
-                // lsn will be lost.
-                if (lastFlushedLSN.get() >= retVal) {
-                    logPages[pageIndex].decRefCnt();
-                    continue;
-                }
-            }
-
-            return retVal;
-        }
-    }
-
-    @Override
-    public void log(byte logType, ITransactionContext txnCtx, int datasetId, int PKHashValue, long resourceId,
-            byte resourceMgrId, int logContentSize, ReusableLogContentObject reusableLogContentObject, ILogger logger,
-            LogicalLogLocator logicalLogLocator) throws ACIDException {
-
-        HashMap<ITransactionContext, Integer> map = null;
-        int activeTxnCount;
-
-        // logLocator is a re-usable object that is appropriately set in each
-        // invocation.
-        // If the reference is null, the log manager must throw an exception.
-        if (logicalLogLocator == null) {
-            throw new ACIDException(
-                    " you need to pass in a non-null logLocator, if you dont have it, then pass in a dummy so that the +"
-                            + "log manager can set it approporiately for you");
-        }
-
-        // compute the total log size including the header and the checksum.
-        int totalLogSize = logRecordHelper.getLogRecordSize(logType, logContentSize);
-
-        // check for the total space requirement to be less than a log page.
-        if (totalLogSize > logPageSize) {
-            throw new ACIDException(
-                    " Maximum Log Content Size is "
-                            + (logPageSize - logRecordHelper.getLogHeaderSize(LogType.UPDATE) - logRecordHelper
-                                    .getLogChecksumSize()));
-        }
-
-        // all constraints checked and we are good to go and acquire a lsn.
-        long previousLSN = -1;
-
-        // the will be set to the location (a long value) where the log record
-        // needs to be placed.
-        long currentLSN;
-
-        // The logs written by a transaction need to be linked to each other for
-        // a successful rollback/recovery. However there could be multiple
-        // threads operating concurrently that are part of a common transaction.
-        // These threads need to synchronize and record the lsn corresponding to
-        // the last log record written by (any thread of) the transaction.
-        synchronized (txnCtx) {
-            previousLSN = txnCtx.getLastLogLocator().getLsn();
-            currentLSN = getLsn(totalLogSize, logType);
-            txnCtx.setLastLSN(currentLSN);
-            if (IS_DEBUG_MODE) {
-                System.out.println("--------------> LSN(" + currentLSN + ") is allocated");
-            }
-            logicalLogLocator.setLsn(currentLSN);
-        }
-
-        /*
-         * At this point, the transaction thread has obtained reserved space for
-         * writing the log record. In doing so, it has acquired (shared)
-         * ownership of the log page. All subsequent actions are under a try
-         * catch block so that if any exception is encountered, a clean can be
-         * performed correctly that is ownership is released.
-         */
-
-        // indicates if the transaction thread has release ownership of the
-        // page.
-        boolean decremented = false;
-
-        int pageIndex = (int) getLogPageIndex(currentLSN);
-
-        // the lsn has been obtained for the log record. need to set the
-        // LogLocator instance accordingly.
-        try {
-            logicalLogLocator.setBuffer(logPages[pageIndex]);
-            int pageOffset = getLogPageOffset(currentLSN);
-            logicalLogLocator.setMemoryOffset(pageOffset);
-
-            // write the log header.
-            logRecordHelper.writeLogHeader(logicalLogLocator, logType, txnCtx, datasetId, PKHashValue, previousLSN,
-                    resourceId, resourceMgrId, logContentSize);
-
-            // increment the offset so that the transaction can fill up the
-            // content in the correct region of the allocated space.
-            logicalLogLocator.increaseMemoryOffset(logRecordHelper.getLogHeaderSize(logType));
-
-            // a COMMIT log record does not have any content and hence
-            // the logger (responsible for putting the log content) is not
-            // invoked.
-            if (logContentSize != 0) {
-                logger.preLog(txnCtx, reusableLogContentObject);
-            }
-
-            if (logContentSize != 0) {
-                // call the logger implementation and ask to fill in the log
-                // record content at the allocated space.
-                logger.log(txnCtx, logicalLogLocator, logContentSize, reusableLogContentObject);
-                logger.postLog(txnCtx, reusableLogContentObject);
-                if (IS_DEBUG_MODE) {
-                    logicalLogLocator.setMemoryOffset(logicalLogLocator.getMemoryOffset()
-                            - logRecordHelper.getLogHeaderSize(logType));
-                    System.out.println(logRecordHelper.getLogRecordForDisplay(logicalLogLocator));
-                    logicalLogLocator.increaseMemoryOffset(logRecordHelper.getLogHeaderSize(logType));
-                }
-            }
-
-            // The log record has been written. For integrity checks, compute
-            // the checksum and put it at the end of the log record.
-            int startPosChecksum = logicalLogLocator.getMemoryOffset() - logRecordHelper.getLogHeaderSize(logType);
-            int length = totalLogSize - logRecordHelper.getLogChecksumSize();
-            long checksum = DataUtil.getChecksum(logPages[pageIndex], startPosChecksum, length);
-            logPages[pageIndex].writeLong(pageOffset + logRecordHelper.getLogHeaderSize(logType) + logContentSize,
-                    checksum);
-
-            // forward the nextWriteOffset in the log page
-            int bufferNextWriteOffset = (int) ((currentLSN + totalLogSize) % logPageSize);
-            if (bufferNextWriteOffset == 0) {
-                bufferNextWriteOffset = logPageSize;
-            }
-            logPages[pageIndex].setBufferNextWriteOffset(bufferNextWriteOffset);
-
-            if (logType != LogType.ENTITY_COMMIT) {
-                if (logType == LogType.COMMIT && txnCtx.isExlusiveJobLevelCommit()) {
-                    synchronized (this) {
-                        map = activeTxnCountMaps.get(pageIndex);
-                        map.put(txnCtx, 1);
-                    }
-                }
-                // release the ownership as the log record has been placed in
-                // created space.
-                logPages[pageIndex].decRefCnt();
-
-                // indicating that the transaction thread has released ownership
-                decremented = true;
-            }
-
-            if (logType == LogType.ENTITY_COMMIT) {
-                synchronized (this) {
-                    map = activeTxnCountMaps.get(pageIndex);
-                    if (map.containsKey(txnCtx)) {
-                        activeTxnCount = (Integer) map.get(txnCtx);
-                        activeTxnCount++;
-                        map.put(txnCtx, activeTxnCount);
-                    } else {
-                        map.put(txnCtx, 1);
-                    }
-                }
-                // ------------------------------------------------------------------------------
-                // [Notice]
-                // reference count should be decremented
-                // after activeTxnCount is incremented, but before
-                // addFlushRequest() is called.
-                // ------------------------------------------------------------------------------
-
-                // release the ownership as the log record has been placed in
-                // created space.
-                logPages[pageIndex].decRefCnt();
-
-                // indicating that the transaction thread has released ownership
-                decremented = true;
-
-                addFlushRequest(pageIndex, currentLSN, false);
-            } else if (logType == LogType.COMMIT) {
-
-                addFlushRequest(pageIndex, currentLSN, true);
-                if (IS_DEBUG_MODE) {
-                    System.out.println("Running sum of log size: " + statLogSize + ", log count: " + statLogCount);
-                }
-            }
-
-            if (IS_DEBUG_MODE) {
-                System.out.println("--------------> LSN(" + currentLSN + ") is written");
-            }
-
-            // collect statistics
-            statLogSize += totalLogSize;
-            statLogCount++;
-
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new ACIDException(txnCtx, "Thread: " + Thread.currentThread().getName()
-                    + " logger encountered exception", e);
-        } finally {
-            if (!decremented) {
-                logPages[pageIndex].decRefCnt();
-            }
-        }
-    }
-
-    /*
-     * This method resets the log page and is called by the log flusher thread
-     * after a page has been flushed to disk.
-     */
-    public void resetLogPage(long lsn, long nextWritePosition, int pageIndex) throws IOException {
-
-        String filePath = LogUtil.getLogFilePath(logManagerProperties, getLogFileId(lsn));
-
-        logPages[pageIndex].reset(filePath, LogUtil.getFileOffset(this, nextWritePosition), logPageSize);
-    }
-
-    @Override
-    public ILogCursor readLog(PhysicalLogLocator physicalLogLocator, ILogFilter logFilter) throws IOException,
-            ACIDException {
-        LogCursor cursor = new LogCursor(this, physicalLogLocator, logFilter, logPageSize);
-        return cursor;
-    }
-
-    /*
-     * Read a log that is residing on the disk.
-     */
-    private void readDiskLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
-        String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, lsnValue));
-        long fileOffset = LogUtil.getFileOffset(this, lsnValue);
-
-        ByteBuffer buffer = ByteBuffer.allocate(logPageSize);
-        RandomAccessFile raf = null;
-        FileChannel fileChannel = null;
-        try {
-            raf = new RandomAccessFile(filePath, "r");
-            fileChannel = raf.getChannel();
-            fileChannel.position(fileOffset);
-            fileChannel.read(buffer);
-            buffer.position(0);
-
-            byte logType = buffer.get(4);
-            int logHeaderSize = logRecordHelper.getLogHeaderSize(logType);
-            int logBodySize = buffer.getInt(logHeaderSize - 4);
-            int logRecordSize = logHeaderSize + logBodySize + logRecordHelper.getLogChecksumSize();
-            buffer.limit(logRecordSize);
-            MemBasedBuffer memBuffer = new MemBasedBuffer(buffer.slice());
-            if (logicalLogLocator == null) {
-                logicalLogLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
-            } else {
-                logicalLogLocator.setLsn(lsnValue);
-                logicalLogLocator.setBuffer(memBuffer);
-                logicalLogLocator.setMemoryOffset(0);
-            }
-            if (!logRecordHelper.validateLogRecord(logicalLogLocator)) {
-                throw new ACIDException(" invalid log record at lsn " + lsnValue);
-            }
-        } catch (Exception fnfe) {
-            fnfe.printStackTrace();
-            throw new ACIDException(" unable to retrieve log record with lsn " + lsnValue + " from the file system",
-                    fnfe);
-        } finally {
-            try {
-                if (fileChannel != null) {
-                    fileChannel.close();
-                } else if (raf != null) {
-                    raf.close();
-                }
-            } catch (IOException ioe) {
-                ioe.printStackTrace();
-                throw new ACIDException(" exception in closing a file: " + filePath, ioe);
-            }
-        }
-    }
-
-    @Override
-    public void readLog(long lsnValue, LogicalLogLocator logicalLogLocator) throws ACIDException {
-        byte[] logRecord = null;
-
-        if (lsnValue >= lsn.get()) {
-            throw new ACIDException(" invalid lsn " + lsnValue);
-        }
-
-        /* check if the log record in the log buffer or has reached the disk. */
-        if (isMemoryRead(lsnValue)) {
-            int pageIndex = getLogPageIndex(lsnValue);
-            int pageOffset = getLogPageOffset(lsnValue);
-
-            // TODO
-            // minimize memory allocation overhead. current code allocates the
-            // log page size per reading a log record.
-
-            byte[] pageContent = new byte[logPageSize];
-
-            // take a lock on the log page so that the page is not flushed to
-            // disk interim
-            synchronized (logPages[pageIndex]) {
-
-                // need to check again (this thread may have got de-scheduled
-                // and must refresh!)
-
-                if (isMemoryRead(lsnValue)) {
-                    // get the log record length
-                    logPages[pageIndex].getBytes(pageContent, 0, pageContent.length);
-                    byte logType = pageContent[pageOffset + 4];
-                    int logHeaderSize = logRecordHelper.getLogHeaderSize(logType);
-                    int logBodySize = DataUtil.byteArrayToInt(pageContent, pageOffset + logHeaderSize - 4);
-                    int logRecordSize = logHeaderSize + logBodySize + logRecordHelper.getLogChecksumSize();
-                    logRecord = new byte[logRecordSize];
-
-                    // copy the log record content
-                    System.arraycopy(pageContent, pageOffset, logRecord, 0, logRecordSize);
-                    MemBasedBuffer memBuffer = new MemBasedBuffer(logRecord);
-                    if (logicalLogLocator == null) {
-                        logicalLogLocator = new LogicalLogLocator(lsnValue, memBuffer, 0, this);
-                    } else {
-                        logicalLogLocator.setLsn(lsnValue);
-                        logicalLogLocator.setBuffer(memBuffer);
-                        logicalLogLocator.setMemoryOffset(0);
-                    }
-                    try {
-                        // validate the log record by comparing checksums
-                        if (!logRecordHelper.validateLogRecord(logicalLogLocator)) {
-                            throw new ACIDException(" invalid log record at lsn " + lsnValue);
-                        }
-                    } catch (Exception e) {
-                        throw new ACIDException("exception encoutered in validating log record at lsn " + lsnValue, e);
-                    }
-                    return;
-                }
-            }
-        }
-
-        // the log record is residing on the disk, read it from there.
-        readDiskLog(lsnValue, logicalLogLocator);
-    }
-
-    public boolean isMemoryRead(long readLSN) {
-        long flushLSN = lastFlushedLSN.get();
-        if ((flushLSN + 1) == readLSN) {
-            return false;
-        }
-        long logPageBeginOffset = flushLSN - (flushLSN % logPageSize);
-        long logPageEndOffset = logPageBeginOffset + logPageSize;
-        if (readLSN > flushLSN || (readLSN >= logPageBeginOffset && readLSN < logPageEndOffset)) {
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    public void renewLogFiles() throws ACIDException {
-        List<String> logFileNames = LogUtil.getLogFiles(logManagerProperties);
-        for (String name : logFileNames) {
-            File file = new File(LogUtil.getLogFilePath(logManagerProperties, Long.parseLong(name)));
-            if (!file.delete()) {
-                throw new ACIDException("Failed to delete a file: " + name);
-            }
-        }
-        closeLogPages();
-        initLSN();
-        openLogPages();
-        logPageFlusher.renew();
-    }
-
-    private void initLSN() throws ACIDException {
-        PhysicalLogLocator nextPhysicalLsn = LogUtil.initializeLogAnchor(this);
-        startingLSN = nextPhysicalLsn.getLsn();
-        lastFlushedLSN.set(startingLSN - 1);
+        appendLSN = initializeLogAnchor();
+        flushLSN.set(appendLSN);
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" Starting lsn is : " + startingLSN);
+            LOGGER.info("LogManager starts logging in LSN: " + appendLSN);
         }
-        lsn.set(startingLSN);
-    }
-
-    private void closeLogPages() throws ACIDException {
-        for (int i = 0; i < numLogPages; i++) {
-            try {
-                logPages[i].close();
-            } catch (IOException e) {
-                throw new ACIDException(e);
-            }
-        }
-    }
-
-    private void openLogPages() throws ACIDException {
-        try {
-            String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, startingLSN));
-            for (int i = 0; i < numLogPages; i++) {
-                logPages[i].open(filePath, LogUtil.getFileOffset(this, startingLSN) + i * logPageSize, logPageSize);
-            }
-        } catch (Exception e) {
-            throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
-        }
+        appendChannel = getFileChannel(appendLSN, false);
+        getAndInitNewPage();
+        logFlusher = new LogFlusher(this, emptyQ, flushQ);
+        logFlusher.setDaemon(true);
+        AsterixThreadExecutor.INSTANCE.execute(logFlusher);
     }
 
     @Override
-    public ILogRecordHelper getLogRecordHelper() {
-        return logRecordHelper;
-    }
-
-    /*
-     * Map each log page to cover a physical byte range over a log file. When a
-     * page is flushed, the page contents are put to disk in the corresponding
-     * byte range.
-     */
-    private void initializeLogPages(long beginLsn) throws ACIDException {
-        try {
-            String filePath = LogUtil.getLogFilePath(logManagerProperties, LogUtil.getFileId(this, beginLsn));
-            long nextDiskWriteOffset = LogUtil.getFileOffset(this, beginLsn);
-            long nextBufferWriteOffset = nextDiskWriteOffset % logPageSize;
-            long bufferBeginOffset = nextDiskWriteOffset - nextBufferWriteOffset;
-
-            for (int i = 0; i < numLogPages; i++) {
-                logPages[i] = FileUtil.getFileBasedBuffer(filePath, bufferBeginOffset + i * logPageSize, logPageSize,
-                        logManagerProperties.getDiskSectorSize());
-                if (i == 0) {
-                    logPages[i].setBufferLastFlushOffset((int) nextBufferWriteOffset);
-                    logPages[i].setBufferNextWriteOffset((int) nextBufferWriteOffset);
-                    logPages[i].setDiskNextWriteOffset(nextDiskWriteOffset);
+    public void log(ILogRecord logRecord) {
+        if (logRecord.getLogSize() > logPageSize) {
+            throw new IllegalStateException();
+        }
+        syncLog(logRecord);
+        if (logRecord.getLogType() == LogType.JOB_COMMIT && !logRecord.isFlushed()) {
+            synchronized (logRecord) {
+                while (!logRecord.isFlushed()) {
+                    try {
+                        logRecord.wait();
+                    } catch (InterruptedException e) {
+                        //ignore
+                    }
                 }
             }
-        } catch (Exception e) {
-            e.printStackTrace();
-            throw new ACIDException(Thread.currentThread().getName() + " unable to create log buffer", e);
         }
     }
 
-    /*
-     * Pages are sequenced starting with 0 going upto numLogPages-1.
-     */
-    public int getNextPageInSequence(int pageNo) {
-        return (pageNo + 1) % numLogPages;
+    private synchronized void syncLog(ILogRecord logRecord) {
+        ITransactionContext txnCtx = logRecord.getTxnCtx();
+        if (getLogFileOffset(appendLSN) + logRecord.getLogSize() > logFileSize) {
+            prepareNextLogFile();
+            appendPage.isFull(true);
+            getAndInitNewPage();
+        } else if (!appendPage.hasSpace(logRecord.getLogSize(), getLogFileOffset(appendLSN))) {
+            appendPage.isFull(true);
+            getAndInitNewPage();
+        }
+        if (logRecord.getLogType() == LogType.UPDATE) {
+            logRecord.setPrevLSN(txnCtx.getLastLSN());
+        }
+        appendPage.append(logRecord, appendLSN);
+        appendLSN += logRecord.getLogSize();
     }
 
-    public int getPreviousPageInSequence(int pageNo) {
-        return pageNo == 0 ? numLogPages - 1 : pageNo - 1;
+    private void getAndInitNewPage() {
+        appendPage = null;
+        while (appendPage == null) {
+            try {
+                appendPage = emptyQ.take();
+            } catch (InterruptedException e) {
+                //ignore
+            }
+        }
+        appendPage.reset();
+        appendPage.setFileChannel(appendChannel);
+        appendPage.setInitialFlushOffset(getLogFileOffset(appendLSN));
+        flushQ.offer(appendPage);
+    }
+
+    private void prepareNextLogFile() {
+        appendLSN += logFileSize - getLogFileOffset(appendLSN);
+        appendChannel = getFileChannel(appendLSN, true);
+        appendPage.isLastPage(true);
+        //[Notice]
+        //the current log file channel is closed if 
+        //LogPage.flush() completely flush the last page of the file.
     }
 
     @Override
+    public ILogReader getLogReader(boolean isRecoveryMode) {
+        return new LogReader(this, logFileSize, logPageSize, flushLSN, isRecoveryMode);
+    }
+
     public LogManagerProperties getLogManagerProperties() {
         return logManagerProperties;
     }
 
-    public IFileBasedBuffer getLogPage(int pageIndex) {
-        return logPages[pageIndex];
-    }
-
-    public IFileBasedBuffer[] getLogPages() {
-        return logPages;
-    }
-
-    @Override
     public TransactionSubsystem getTransactionSubsystem() {
-        return provider;
+        return txnSubsystem;
     }
 
-    static AtomicInteger t = new AtomicInteger();
-
-    public synchronized void decrementActiveTxnCountOnIndexes(int pageIndex) throws HyracksDataException {
-        ITransactionContext ctx = null;
-        int count = 0;
-        int i = 0;
-
-        HashMap<ITransactionContext, Integer> map = activeTxnCountMaps.get(pageIndex);
-        Set<Map.Entry<ITransactionContext, Integer>> entrySet = map.entrySet();
-        if (entrySet != null) {
-            for (Map.Entry<ITransactionContext, Integer> entry : entrySet) {
-                if (entry != null) {
-                    if (entry.getValue() != null) {
-                        count = entry.getValue();
-                    }
-                    if (count > 0) {
-                        ctx = entry.getKey();
-                        for (i = 0; i < count; i++) {
-                            ctx.decreaseActiveTransactionCountOnIndexes();
-                        }
-                    }
-                }
-            }
-        }
-
-        map.clear();
+    public long getAppendLSN() {
+        return appendLSN;
     }
 
     @Override
@@ -783,9 +208,8 @@
         try {
             StringBuilder sb = new StringBuilder();
             sb.append("\n>>dump_begin\t>>----- [LSNInfo] -----");
-            sb.append("\nstartingLSN: " + startingLSN);
-            sb.append("\ncurrentLSN: " + lsn.get());
-            sb.append("\nlastFlushedLSN: " + lastFlushedLSN.get());
+            sb.append("\nappendLsn: " + appendLSN);
+            sb.append("\nflushLsn: " + flushLSN.get());
             sb.append("\n>>dump_end\t>>----- [LSNInfo] -----\n");
             os.write(sb.toString().getBytes());
         } catch (Exception e) {
@@ -795,179 +219,195 @@
             }
         }
     }
-}
 
-/*
- * The thread responsible for putting log pages to disk in an ordered manner.
- * The Log Flusher updates the bookkeeping data internal to the log manager and
- * acquires appropriate locks. It also acquires finer level locks on the log
- * page when it is in process of flushing the content to disk.
- */
-class LogPageFlushThread extends Thread {
-
-    private LogManager logManager;
-    /*
-     * pendingFlushRequests is a map with key as Integer denoting the page
-     * index. When a (transaction) thread discovers the need to flush a page, it
-     * puts its Thread object into the corresponding value that is a
-     * LinkedBlockingQueue. The LogManager has a LogFlusher thread that scans
-     * this map in order of page index (and circling around). The flusher thread
-     * needs to flush pages in order and waits for a thread to deposit an object
-     * in the blocking queue corresponding to the next page in order. A request
-     * to flush a page is conveyed to the flush thread by simply depositing an
-     * object in to corresponding blocking queue. It is blocking in the sense
-     * that the flusher thread will continue to wait for an object to arrive in
-     * the queue. The object itself is ignored by the fliusher and just acts as
-     * a signal/event that a page needs to be flushed.
-     */
-    private final LinkedBlockingQueue<Object>[] flushRequestQueue;
-    private final Object[] flushRequests;
-    private int flushPageIndex;
-    private final long groupCommitWaitPeriod;
-    private boolean isRenewRequest;
-
-    public LogPageFlushThread(LogManager logManager) {
-        this.logManager = logManager;
-        setName("Flusher");
-        int numLogPages = logManager.getLogManagerProperties().getNumLogPages();
-        this.flushRequestQueue = new LinkedBlockingQueue[numLogPages];
-        this.flushRequests = new Object[numLogPages];
-        for (int i = 0; i < numLogPages; i++) {
-            flushRequestQueue[i] = new LinkedBlockingQueue<Object>(1);
-            flushRequests[i] = new Object();
-        }
-        this.flushPageIndex = 0;
-        groupCommitWaitPeriod = logManager.getLogManagerProperties().getGroupCommitWaitPeriod();
-        isRenewRequest = false;
+    public MutableLong getFlushLSN() {
+        return flushLSN;
     }
 
-    public void renew() {
-        isRenewRequest = true;
-        flushPageIndex = 0;
-        this.interrupt();
-        isRenewRequest = false;
-    }
-
-    public void requestFlush(int pageIndex, long lsn, boolean isSynchronous) {
-        synchronized (logManager.getLogPage(pageIndex)) {
-            // return if flushedLSN >= lsn
-            if (logManager.getLastFlushedLsn().get() >= lsn) {
-                return;
-            }
-
-            // put a new request to the queue only if the request on the page is
-            // not in the queue.
-            flushRequestQueue[pageIndex].offer(flushRequests[pageIndex]);
-
-            // return if the request is asynchronous
-            if (!isSynchronous) {
-                return;
-            }
-
-            // wait until there is flush.
-            boolean isNotified = false;
-            while (!isNotified) {
-                try {
-                    logManager.getLogPage(pageIndex).wait();
-                    isNotified = true;
-                } catch (InterruptedException e) {
-                    e.printStackTrace();
+    private long initializeLogAnchor() {
+        long fileId = 0;
+        long offset = 0;
+        File fileLogDir = new File(logDir);
+        try {
+            if (fileLogDir.exists()) {
+                List<Long> logFileIds = getLogFileIds();
+                if (logFileIds == null) {
+                    createFileIfNotExists(getLogFilePath(0));
+                    if (LOGGER.isLoggable(Level.INFO)) {
+                        LOGGER.info("created a log file: " + getLogFilePath(0));
+                    }
+                } else {
+                    fileId = logFileIds.get(logFileIds.size() - 1);
+                    File logFile = new File(getLogFilePath(fileId));
+                    offset = logFile.length();
+                }
+            } else {
+                createNewDirectory(logDir);
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("created the log directory: " + logManagerProperties.getLogDir());
+                }
+                createFileIfNotExists(getLogFilePath(0));
+                if (LOGGER.isLoggable(Level.INFO)) {
+                    LOGGER.info("created a log file: " + getLogFilePath(0));
                 }
             }
+        } catch (IOException ioe) {
+            throw new IllegalStateException("Failed to initialize the log anchor", ioe);
         }
+        if (LOGGER.isLoggable(Level.INFO)) {
+            LOGGER.info("log file Id: " + fileId + ", offset: " + offset);
+        }
+        return logFileSize * fileId + offset;
+    }
+
+    public void renewLogFiles() {
+        terminateLogFlusher();
+        deleteAllLogFiles();
+        initializeLogManager();
+    }
+
+    private void terminateLogFlusher() {
+        logFlusher.terminate();
+        try {
+            logFlusher.join();
+        } catch (InterruptedException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    private void deleteAllLogFiles() {
+        List<Long> logFileIds = getLogFileIds();
+        for (Long id : logFileIds) {
+            File file = new File(getLogFilePath(id));
+            if (!file.delete()) {
+                throw new IllegalStateException("Failed to delete a file: " + file.getAbsolutePath());
+            }
+        }
+    }
+
+    private List<Long> getLogFileIds() {
+        File fileLogDir = new File(logDir);
+        String[] logFileNames = null;
+        List<Long> logFileIds = null;
+        if (fileLogDir.exists()) {
+            logFileNames = fileLogDir.list(new FilenameFilter() {
+                public boolean accept(File dir, String name) {
+                    if (name.startsWith(logFilePrefix)) {
+                        return true;
+                    }
+                    return false;
+                }
+            });
+            if (logFileNames != null && logFileNames.length != 0) {
+                logFileIds = new ArrayList<Long>();
+                for (String fileName : logFileNames) {
+                    logFileIds.add(Long.parseLong(fileName.substring(logFilePrefix.length() + 1)));
+                }
+                Collections.sort(logFileIds, new Comparator<Long>() {
+                    @Override
+                    public int compare(Long arg0, Long arg1) {
+                        if (arg0 > arg1) {
+                            return 1;
+                        } else if (arg0 == arg1) {
+                            return 0;
+                        } else {
+                            return -1;
+                        }
+                    }
+                });
+            }
+        }
+        return logFileIds;
+    }
+
+    public String getLogFilePath(long fileId) {
+        return logDir + File.separator + logFilePrefix + "_" + fileId;
+    }
+
+    public long getLogFileOffset(long lsn) {
+        return lsn % logFileSize;
+    }
+
+    public long getLogFileId(long lsn) {
+        return lsn / logFileSize;
+    }
+
+    private boolean createFileIfNotExists(String path) throws IOException {
+        File file = new File(path);
+        File parentFile = file.getParentFile();
+        if (parentFile != null) {
+            parentFile.mkdirs();
+        }
+        return file.createNewFile();
+    }
+
+    private boolean createNewDirectory(String path) throws IOException {
+        return (new File(path)).mkdir();
+    }
+
+    public FileChannel getFileChannel(long lsn, boolean create) {
+        FileChannel newFileChannel = null;
+        try {
+            long fileId = getLogFileId(lsn);
+            String logFilePath = getLogFilePath(fileId);
+            File file = new File(logFilePath);
+            if (create) {
+                if (!file.createNewFile()) {
+                    throw new IllegalStateException();
+                }
+            } else {
+                if (!file.exists()) {
+                    throw new IllegalStateException();
+                }
+            }
+            RandomAccessFile raf = new RandomAccessFile(new File(logFilePath), "rw");
+            newFileChannel = raf.getChannel();
+            newFileChannel.position(getLogFileOffset(lsn));
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+        return newFileChannel;
+    }
+}
+
+class LogFlusher extends Thread {
+    private final LogManager logMgr;
+    private final LinkedBlockingQueue<LogPage> emptyQ;
+    private final LinkedBlockingQueue<LogPage> flushQ;
+    private LogPage flushPage;
+    private boolean stop;
+
+    public LogFlusher(LogManager logMgr, LinkedBlockingQueue<LogPage> emptyQ, LinkedBlockingQueue<LogPage> flushQ) {
+        this.logMgr = logMgr;
+        this.emptyQ = emptyQ;
+        this.flushQ = flushQ;
+        flushPage = null;
+        stop = false;
+    }
+
+    public void terminate() {
+        if (flushPage != null) {
+            flushPage.isStop(true);
+        }
+        stop = true;
+        this.interrupt();
     }
 
     @Override
     public void run() {
-        int logPageSize = logManager.getLogManagerProperties().getLogPageSize();
-        int logBufferSize = logManager.getLogManagerProperties().getLogBufferSize();
-        int beforeFlushOffset = 0;
-        int afterFlushOffset = 0;
-        boolean resetFlushPageIndex = false;
-
         while (true) {
+            flushPage = null;
             try {
-                // A wait call on the linkedBLockingQueue. The flusher thread is
-                // notified when an object is added to the queue. Please note
-                // that each page has an associated blocking queue.
-                try {
-                    flushRequestQueue[flushPageIndex].take();
-                } catch (InterruptedException ie) {
-                    while (isRenewRequest) {
-                        sleep(1);
-                    }
-                    continue;
-                }
-                // if the log page is already full, don't wait.
-                if (logManager.getLogPage(flushPageIndex).getBufferNextWriteOffset() < logPageSize
-                        - logManager.getLogRecordHelper().getCommitLogSize()) {
-                    // #. sleep for the groupCommitWaitTime
-                    sleep(groupCommitWaitPeriod);
-                }
-
-                synchronized (logManager.getLogPage(flushPageIndex)) {
-                    logManager.getLogPage(flushPageIndex).acquireWriteLatch();
-                    try {
-
-                        // #. need to wait until the reference count reaches 0
-                        while (logManager.getLogPage(flushPageIndex).getRefCnt() != 0) {
-                            sleep(0);
-                        }
-
-                        beforeFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
-
-                        // put the content to disk (the thread still has a lock
-                        // on the log page)
-                        logManager.getLogPage(flushPageIndex).flush();
-
-                        afterFlushOffset = logManager.getLogPage(flushPageIndex).getBufferLastFlushOffset();
-
-                        // increment the last flushed lsn
-                        logManager.incrementLastFlushedLsn(afterFlushOffset - beforeFlushOffset);
-
-                        // increment currentLSN if currentLSN is less than
-                        // flushLSN.
-                        if (logManager.getLastFlushedLsn().get() + 1 > logManager.getCurrentLsn().get()) {
-                            logManager.getCurrentLsn().set(logManager.getLastFlushedLsn().get() + 1);
-                        }
-
-                        // Map the log page to a new region in the log file if
-                        // the flushOffset reached the logPageSize
-                        if (afterFlushOffset == logPageSize) {
-                            long diskNextWriteOffset = logManager.getLogPages()[flushPageIndex]
-                                    .getDiskNextWriteOffset() + logBufferSize;
-                            logManager.resetLogPage(logManager.getLastFlushedLsn().get() + 1 + logBufferSize,
-                                    diskNextWriteOffset, flushPageIndex);
-                            resetFlushPageIndex = true;
-                        }
-                    } finally {
-                        logManager.getLogPage(flushPageIndex).releaseWriteLatch();
-                    }
-                    // decrement activeTxnCountOnIndexes
-                    logManager.decrementActiveTxnCountOnIndexes(flushPageIndex);
-
-                    // #. checks the queue whether there is another flush
-                    // request on the same log buffer
-                    // If there is another request, then simply remove it.
-                    if (flushRequestQueue[flushPageIndex].peek() != null) {
-                        flushRequestQueue[flushPageIndex].take();
-                    }
-
-                    // notify all waiting (transaction) threads.
-                    logManager.getLogPage(flushPageIndex).notifyAll();
-
-                    if (resetFlushPageIndex) {
-                        flushPageIndex = logManager.getNextPageInSequence(flushPageIndex);
-                        resetFlushPageIndex = false;
-                    }
-                }
-            } catch (IOException ioe) {
-                ioe.printStackTrace();
-                throw new Error(" exception in flushing log page", ioe);
+                flushPage = flushQ.take();
+                flushPage.flush();
             } catch (InterruptedException e) {
-                e.printStackTrace();
-                break;
+                if (stop) {
+                    break;
+                } else {
+                    throw new IllegalStateException(e);
+                }
             }
+            emptyQ.offer(flushPage);
         }
     }
 }
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
new file mode 100644
index 0000000..5c051a3
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPage.java
@@ -0,0 +1,203 @@
+/*
+ * Copyright 2009-2013 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.transaction.management.service.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogPage;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
+
+public class LogPage implements ILogPage {
+    private final LockManager lockMgr;
+    private final LogPageReader logPageReader;
+    private final int logPageSize;
+    private final MutableLong flushLSN;
+    private final AtomicBoolean full;
+    private int appendOffset;
+    private int flushOffset;
+    private final ByteBuffer appendBuffer;
+    private final ByteBuffer flushBuffer;
+    private final ByteBuffer unlockBuffer;
+    private boolean isLastPage;
+    private final LinkedBlockingQueue<ILogRecord> syncCommitQ;
+    private FileChannel fileChannel;
+    private boolean stop;
+
+    public LogPage(LockManager lockMgr, int logPageSize, MutableLong flushLSN) {
+        this.lockMgr = lockMgr;
+        this.logPageSize = logPageSize;
+        this.flushLSN = flushLSN;
+        appendBuffer = ByteBuffer.allocate(logPageSize);
+        flushBuffer = appendBuffer.duplicate();
+        unlockBuffer = appendBuffer.duplicate();
+        logPageReader = getLogPageReader();
+        full = new AtomicBoolean(false);
+        appendOffset = 0;
+        flushOffset = 0;
+        isLastPage = false;
+        syncCommitQ = new LinkedBlockingQueue<ILogRecord>(logPageSize / ILogRecord.COMMIT_LOG_SIZE);
+    }
+
+    ////////////////////////////////////
+    // LogAppender Methods
+    ////////////////////////////////////
+
+    @Override
+    public void append(ILogRecord logRecord, long appendLSN) {
+        logRecord.writeLogRecord(appendBuffer);
+        if (logRecord.getLogType() == LogType.UPDATE) {
+            logRecord.getTxnCtx().setLastLSN(logRecord.getResourceId(), appendLSN);
+        }
+        synchronized (this) {
+            appendOffset += logRecord.getLogSize();
+            if (logRecord.getLogType() == LogType.JOB_COMMIT) {
+                logRecord.isFlushed(false);
+                syncCommitQ.offer(logRecord);
+            }
+            this.notify();
+        }
+    }
+
+    public void setFileChannel(FileChannel fileChannel) {
+        this.fileChannel = fileChannel;
+    }
+
+    public void setInitialFlushOffset(long offset) {
+        try {
+            fileChannel.position(offset);
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    public synchronized void isFull(boolean full) {
+        this.full.set(full);
+        this.notify();
+    }
+
+    public void isLastPage(boolean isLastPage) {
+        this.isLastPage = isLastPage;
+    }
+
+    public boolean hasSpace(int logSize, long logFileOffset) {
+        return appendOffset + logSize <= logPageSize;
+    }
+
+    public void reset() {
+        appendBuffer.position(0);
+        appendBuffer.limit(logPageSize);
+        flushBuffer.position(0);
+        flushBuffer.limit(logPageSize);
+        unlockBuffer.position(0);
+        unlockBuffer.limit(logPageSize);
+        full.set(false);
+        appendOffset = 0;
+        flushOffset = 0;
+        isLastPage = false;
+    }
+
+    ////////////////////////////////////
+    // LogFlusher Methods
+    ////////////////////////////////////
+
+    @Override
+    public void flush() throws InterruptedException {
+        try {
+            int endOffset;
+            while (!full.get()) {
+                synchronized (this) {
+                    if (appendOffset - flushOffset == 0 && !full.get()) {
+                        try {
+                            this.wait();
+                        } catch (InterruptedException e) {
+                            if (stop) {
+                                throw e;
+                            } else {
+                                throw new IllegalStateException(e);
+                            }
+                        }
+                    }
+                    endOffset = appendOffset;
+                }
+                internalFlush(flushOffset, endOffset);
+            }
+            internalFlush(flushOffset, appendOffset);
+            if (isLastPage) {
+                fileChannel.close();
+            }
+        } catch (IOException e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    private void internalFlush(int beginOffset, int endOffset) {
+        try {
+            if (endOffset > beginOffset) {
+                flushBuffer.limit(endOffset);
+                fileChannel.write(flushBuffer);
+                fileChannel.force(false);
+                flushOffset = endOffset;
+                synchronized (flushLSN) {
+                    flushLSN.set(flushLSN.get() + (endOffset - beginOffset));
+                    flushLSN.notifyAll(); //notify to LogReaders if any
+                }
+                batchUnlock(beginOffset, endOffset);
+            }
+        } catch (Exception e) {
+            throw new IllegalStateException(e);
+        }
+    }
+
+    private LogPageReader getLogPageReader() {
+        return new LogPageReader(unlockBuffer);
+    }
+
+    private void batchUnlock(int beginOffset, int endOffset) throws ACIDException {
+        if (endOffset > beginOffset) {
+            logPageReader.initializeScan(beginOffset, endOffset);
+            lockMgr.batchUnlock(this, logPageReader);
+        }
+    }
+
+    public void notifyJobCommitter() {
+        ILogRecord logRecord = null;
+        while (logRecord == null) {
+            try {
+                logRecord = syncCommitQ.take();
+            } catch (InterruptedException e) {
+                //ignore
+            }
+        }
+        synchronized (logRecord) {
+            logRecord.isFlushed(true);
+            logRecord.notifyAll();
+        }
+    }
+
+    public boolean isStop() {
+        return stop;
+    }
+
+    public void isStop(boolean stop) {
+        this.stop = stop;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java
new file mode 100644
index 0000000..9e54abc
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogPageReader.java
@@ -0,0 +1,44 @@
+/*
+ * Copyright 2009-2013 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.transaction.management.service.logging;
+
+import java.nio.ByteBuffer;
+
+public class LogPageReader {
+
+    private final ByteBuffer buffer;
+    private final LogRecord logRecord;
+    private int endOffset;
+
+    public LogPageReader(ByteBuffer buffer) {
+        this.buffer = buffer;
+        logRecord = new LogRecord();
+    }
+
+    public void initializeScan(int beginOffset, int endOffset) {
+        this.endOffset = endOffset;
+        buffer.position(beginOffset);
+    }
+
+    public LogRecord next() {
+        if (buffer.position() == endOffset) {
+            return null;
+        }
+        if (!logRecord.readLogRecord(buffer)) {
+            throw new IllegalStateException();
+        }
+        return logRecord;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java
new file mode 100644
index 0000000..70ae711
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogReader.java
@@ -0,0 +1,178 @@
+/*
+ * Copyright 2009-2013 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.transaction.management.service.logging;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+
+public class LogReader implements ILogReader {
+
+    private final LogManager logMgr;
+    private final long logFileSize;
+    private final int logPageSize;
+    private final MutableLong flushLSN;
+    private final boolean isRecoveryMode;
+    private final ByteBuffer readBuffer;
+    private final ILogRecord logRecord;
+    private long readLSN;
+    private long bufferBeginLSN;
+    private long fileBeginLSN;
+    private FileChannel fileChannel;
+
+    public LogReader(LogManager logMgr, long logFileSize, int logPageSize, MutableLong flushLSN, boolean isRecoveryMode) {
+        this.logMgr = logMgr;
+        this.logFileSize = logFileSize;
+        this.logPageSize = logPageSize;
+        this.flushLSN = flushLSN;
+        this.isRecoveryMode = isRecoveryMode;
+        this.readBuffer = ByteBuffer.allocate(logPageSize);
+        this.logRecord = new LogRecord();
+    }
+
+    @Override
+    public void initializeScan(long beginLSN) throws ACIDException {
+        readLSN = beginLSN;
+        synchronized (flushLSN) {
+            while (readLSN >= flushLSN.get()) {
+                if (isRecoveryMode) {
+                    return;
+                }
+                try {
+                    flushLSN.wait();
+                } catch (InterruptedException e) {
+                    //ignore.
+                }
+            }
+        }
+        getFileChannel();
+        readPage();
+    }
+
+    //for scanning
+    @Override
+    public ILogRecord next() throws ACIDException {
+        synchronized (flushLSN) {
+            while (readLSN >= flushLSN.get()) {
+                if (isRecoveryMode) {
+                    return null;
+                }
+                try {
+                    flushLSN.wait();
+                } catch (InterruptedException e) {
+                    //ignore
+                }
+            }
+        }
+        if (readBuffer.position() == readBuffer.limit() || !logRecord.readLogRecord(readBuffer)) {
+            readNextPage();
+            if (!logRecord.readLogRecord(readBuffer)) {
+                throw new IllegalStateException();
+            }
+        }
+        logRecord.setLSN(readLSN);
+        readLSN += logRecord.getLogSize();
+        return logRecord;
+    }
+
+    private void readNextPage() throws ACIDException {
+        try {
+            if (readLSN % logFileSize == fileChannel.size()) {
+                fileChannel.close();
+                readLSN += logFileSize - (readLSN % logFileSize);
+                getFileChannel();
+            }
+            readPage();
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+    }
+
+    private void readPage() throws ACIDException {
+        int size;
+        readBuffer.position(0);
+        readBuffer.limit(logPageSize);
+        try {
+            fileChannel.position(readLSN % logFileSize);
+            size = fileChannel.read(readBuffer, logPageSize);
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+        readBuffer.position(0);
+        readBuffer.limit(size);
+        bufferBeginLSN = readLSN;
+    }
+
+    //for random reading
+    @Override
+    public ILogRecord read(long LSN) throws ACIDException {
+        readLSN = LSN;
+        synchronized (flushLSN) {
+            while (readLSN >= flushLSN.get()) {
+                try {
+                    flushLSN.wait();
+                } catch (InterruptedException e) {
+                    //ignore
+                }
+            }
+        }
+        try {
+            if (fileChannel == null) {
+                getFileChannel();
+                readPage();
+            } else if (readLSN < fileBeginLSN || readLSN >= fileBeginLSN + fileChannel.size()) {
+                fileChannel.close();
+                getFileChannel();
+                readPage();
+            } else if (readLSN < bufferBeginLSN || readLSN >= bufferBeginLSN + readBuffer.limit()) {
+                readPage();
+            } else {
+                readBuffer.position((int) (readLSN - bufferBeginLSN));
+            }
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+        if (!logRecord.readLogRecord(readBuffer)) {
+            readNextPage();
+            if (!logRecord.readLogRecord(readBuffer)) {
+                throw new IllegalStateException();
+            }
+        }
+        logRecord.setLSN(readLSN);
+        readLSN += logRecord.getLogSize();
+        return logRecord;
+    }
+
+    private void getFileChannel() throws ACIDException {
+        fileChannel = logMgr.getFileChannel(readLSN, false);
+        fileBeginLSN = readLSN;
+    }
+
+    @Override
+    public void close() throws ACIDException {
+        try {
+            if (fileChannel != null) {
+                fileChannel.close();
+            }
+        } catch (IOException e) {
+            throw new ACIDException(e);
+        }
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
new file mode 100644
index 0000000..6f571ab
--- /dev/null
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecord.java
@@ -0,0 +1,420 @@
+/*
+ * Copyright 2009-2013 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.transaction.management.service.logging;
+
+import java.nio.BufferUnderflowException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.zip.CRC32;
+
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
+import edu.uci.ics.asterix.common.transactions.IRecoveryManager.ResourceType;
+import edu.uci.ics.asterix.common.transactions.ITransactionContext;
+import edu.uci.ics.hyracks.dataflow.common.data.accessors.ITupleReference;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleReference;
+import edu.uci.ics.hyracks.storage.am.common.tuples.SimpleTupleWriter;
+
+/*
+ * == LogRecordFormat ==
+ * ---------------------------
+ * [Header1] (13 bytes) : for all log types
+ * LogType(1)
+ * JobId(4)
+ * DatasetId(4) //stored in dataset_dataset in Metadata Node
+ * PKHashValue(4)
+ * ---------------------------
+ * [Header2] (21 bytes) : only for update log type
+ * PrevLSN(8)
+ * ResourceId(8) //stored in .metadata of the corresponding index in NC node
+ * ResourceType(1)
+ * LogRecordSize(4)
+ * ---------------------------
+ * [Body] (Variable size) : only for update log type
+ * FieldCnt(4)
+ * NewOp(1)
+ * NewValueLength(4)
+ * NewValue(NewValueLength)
+ * OldOp(1)
+ * OldValueLength(4)
+ * OldValue(OldValueLength)
+ * ---------------------------
+ * [Tail] (8 bytes) : for all log types
+ * Checksum(8)
+ * ---------------------------
+ * = LogSize =
+ * 1) JOB_COMMIT and ENTITY_COMMIT: 21 bytes
+ * 2) UPDATE: 56 + old and new value size (13 + 21 + 14 + old and newValueSize + 8)
+ */
+public class LogRecord implements ILogRecord {
+
+    //------------- fields in a log record (begin) ------------//
+    private byte logType;
+    private int jobId;
+    private int datasetId;
+    private int PKHashValue;
+    private long prevLSN;
+    private long resourceId;
+    private byte resourceType;
+    private int logSize;
+    private int fieldCnt;
+    private byte newOp;
+    private int newValueSize;
+    private ITupleReference newValue;
+    private byte oldOp;
+    private int oldValueSize;
+    private ITupleReference oldValue;
+    private long checksum;
+    //------------- fields in a log record (end) --------------//
+
+    private static final int CHECKSUM_SIZE = 8;
+    private ITransactionContext txnCtx;
+    private long LSN;
+    private final AtomicBoolean isFlushed;
+    private final SimpleTupleWriter tupleWriter;
+    private final SimpleTupleReference newTuple;
+    private final CRC32 checksumGen;
+
+    public LogRecord() {
+        isFlushed = new AtomicBoolean(false);
+        tupleWriter = new SimpleTupleWriter();
+        newTuple = (SimpleTupleReference) tupleWriter.createTupleReference();
+        checksumGen = new CRC32();
+    }
+
+    @Override
+    public void writeLogRecord(ByteBuffer buffer) {
+        int beginOffset = buffer.position();
+        buffer.put(logType);
+        buffer.putInt(jobId);
+        buffer.putInt(datasetId);
+        buffer.putInt(PKHashValue);
+        if (logType == LogType.UPDATE) {
+            buffer.putLong(prevLSN);
+            buffer.putLong(resourceId);
+            buffer.put(resourceType);
+            buffer.putInt(logSize);
+            buffer.putInt(fieldCnt);
+            buffer.put(newOp);
+            buffer.putInt(newValueSize);
+            writeTuple(buffer, newValue, newValueSize);
+            if (resourceType == ResourceType.LSM_BTREE) {
+                buffer.put(oldOp);
+                if (oldOp != (byte) (IndexOperation.NOOP.ordinal())) {
+                    buffer.putInt(oldValueSize);
+                    if (oldValueSize > 0) {
+                        writeTuple(buffer, oldValue, oldValueSize);
+                    }
+                }
+            }
+        }
+        checksum = generateChecksum(buffer, beginOffset, logSize - CHECKSUM_SIZE);
+        buffer.putLong(checksum);
+    }
+
+    private void writeTuple(ByteBuffer buffer, ITupleReference tuple, int size) {
+        tupleWriter.writeTuple(tuple, buffer.array(), buffer.position());
+        buffer.position(buffer.position() + size);
+    }
+
+    private long generateChecksum(ByteBuffer buffer, int offset, int len) {
+        checksumGen.reset();
+        checksumGen.update(buffer.array(), offset, len);
+        return checksumGen.getValue();
+    }
+
+    @Override
+    public boolean readLogRecord(ByteBuffer buffer) {
+        int beginOffset = buffer.position();
+        try {
+            logType = buffer.get();
+            jobId = buffer.getInt();
+            datasetId = buffer.getInt();
+            PKHashValue = buffer.getInt();
+            if (logType == LogType.UPDATE) {
+                prevLSN = buffer.getLong();
+                resourceId = buffer.getLong();
+                resourceType = buffer.get();
+                logSize = buffer.getInt();
+                fieldCnt = buffer.getInt();
+                newOp = buffer.get();
+                newValueSize = buffer.getInt();
+                newValue = readTuple(buffer, newValueSize);
+                if (resourceType == ResourceType.LSM_BTREE) {
+                    oldOp = buffer.get();
+                    if (oldOp != (byte) (IndexOperation.NOOP.ordinal())) {
+                        oldValueSize = buffer.getInt();
+                        if (oldValueSize > 0) {
+                            oldValue = readTuple(buffer, oldValueSize);
+                        }
+                    }
+                }
+            } else {
+                logSize = COMMIT_LOG_SIZE;
+            }
+            checksum = buffer.getLong();
+            if (checksum != generateChecksum(buffer, beginOffset, logSize - CHECKSUM_SIZE)) {
+                throw new IllegalStateException();
+            }
+        } catch (BufferUnderflowException e) {
+            buffer.position(beginOffset);
+            return false;
+        }
+        return true;
+    }
+
+    private ITupleReference readTuple(ByteBuffer buffer, int size) {
+        newTuple.setFieldCount(fieldCnt);
+        newTuple.resetByTupleOffset(buffer, buffer.position());
+        buffer.position(buffer.position() + size);
+        return newTuple;
+    }
+
+    @Override
+    public void formCommitLogRecord(ITransactionContext txnCtx, byte logType, int jobId, int datasetId, int PKHashValue) {
+        this.txnCtx = txnCtx;
+        this.logType = logType;
+        this.jobId = jobId;
+        this.datasetId = datasetId;
+        this.PKHashValue = PKHashValue;
+        this.logSize = COMMIT_LOG_SIZE;
+    }
+
+    @Override
+    public void setUpdateLogSize() {
+        logSize = UPDATE_LOG_BASE_SIZE + newValueSize + oldValueSize;
+        if (resourceType != ResourceType.LSM_BTREE) {
+            logSize -= 5; //oldOp(byte: 1) + oldValueLength(int: 4)
+        } else {
+            if (oldOp == (byte) (IndexOperation.NOOP.ordinal())) {
+                logSize -= 4; //oldValueLength(int: 4)
+            }
+        }
+    }
+
+    @Override
+    public String getLogRecordForDisplay() {
+        StringBuilder builder = new StringBuilder();
+        String logTypeDisplay = null;
+        switch (logType) {
+            case LogType.JOB_COMMIT:
+                logTypeDisplay = "JOB_COMMIT";
+                break;
+            case LogType.UPDATE:
+                logTypeDisplay = "UPDATE";
+                break;
+            case LogType.ENTITY_COMMIT:
+                logTypeDisplay = "ENTITY_COMMIT";
+                break;
+        }
+        builder.append(" LSN : ").append(LSN);
+        builder.append(" LogType : ").append(logTypeDisplay);
+        builder.append(" JobId : ").append(jobId);
+        builder.append(" DatasetId : ").append(datasetId);
+        builder.append(" PKHashValue : ").append(PKHashValue);
+        if (logType == LogType.UPDATE) {
+            builder.append(" PrevLSN : ").append(prevLSN);
+            builder.append(" ResourceId : ").append(resourceId);
+            builder.append(" ResourceType : ").append(resourceType);
+            builder.append(" LogSize : ").append(logSize);
+        }
+        return builder.toString();
+    }
+
+    ////////////////////////////////////////////
+    // getter and setter methods
+    ////////////////////////////////////////////
+
+    @Override
+    public ITransactionContext getTxnCtx() {
+        return txnCtx;
+    }
+
+    @Override
+    public void setTxnCtx(ITransactionContext txnCtx) {
+        this.txnCtx = txnCtx;
+    }
+
+    @Override
+    public boolean isFlushed() {
+        return isFlushed.get();
+    }
+
+    @Override
+    public void isFlushed(boolean isFlushed) {
+        this.isFlushed.set(isFlushed);
+    }
+
+    @Override
+    public byte getLogType() {
+        return logType;
+    }
+
+    @Override
+    public void setLogType(byte logType) {
+        this.logType = logType;
+    }
+
+    @Override
+    public int getJobId() {
+        return jobId;
+    }
+
+    @Override
+    public void setJobId(int jobId) {
+        this.jobId = jobId;
+    }
+
+    @Override
+    public int getDatasetId() {
+        return datasetId;
+    }
+
+    @Override
+    public void setDatasetId(int datasetId) {
+        this.datasetId = datasetId;
+    }
+
+    @Override
+    public int getPKHashValue() {
+        return PKHashValue;
+    }
+
+    @Override
+    public void setPKHashValue(int PKHashValue) {
+        this.PKHashValue = PKHashValue;
+    }
+
+    @Override
+    public long getPrevLSN() {
+        return prevLSN;
+    }
+
+    @Override
+    public void setPrevLSN(long prevLSN) {
+        this.prevLSN = prevLSN;
+    }
+
+    @Override
+    public long getResourceId() {
+        return resourceId;
+    }
+
+    @Override
+    public void setResourceId(long resourceId) {
+        this.resourceId = resourceId;
+    }
+
+    @Override
+    public byte getResourceType() {
+        return resourceType;
+    }
+
+    @Override
+    public void setResourceType(byte resourceType) {
+        this.resourceType = resourceType;
+    }
+
+    @Override
+    public int getLogSize() {
+        return logSize;
+    }
+
+    @Override
+    public void setLogSize(int logSize) {
+        this.logSize = logSize;
+    }
+
+    @Override
+    public byte getNewOp() {
+        return newOp;
+    }
+
+    @Override
+    public void setNewOp(byte newOp) {
+        this.newOp = newOp;
+    }
+
+    @Override
+    public int getNewValueSize() {
+        return newValueSize;
+    }
+
+    @Override
+    public void setNewValueSize(int newValueSize) {
+        this.newValueSize = newValueSize;
+    }
+
+    @Override
+    public ITupleReference getNewValue() {
+        return newValue;
+    }
+
+    @Override
+    public void setNewValue(ITupleReference newValue) {
+        this.newValue = newValue;
+        this.fieldCnt = newValue.getFieldCount();
+    }
+
+    @Override
+    public byte getOldOp() {
+        return oldOp;
+    }
+
+    @Override
+    public void setOldOp(byte oldOp) {
+        this.oldOp = oldOp;
+    }
+
+    @Override
+    public int getOldValueSize() {
+        return oldValueSize;
+    }
+
+    @Override
+    public void setOldValueSize(int oldValueSize) {
+        this.oldValueSize = oldValueSize;
+    }
+
+    @Override
+    public ITupleReference getOldValue() {
+        return oldValue;
+    }
+
+    @Override
+    public void setOldValue(ITupleReference oldValue) {
+        this.oldValue = oldValue;
+    }
+
+    @Override
+    public long getChecksum() {
+        return checksum;
+    }
+
+    @Override
+    public void setChecksum(long checksum) {
+        this.checksum = checksum;
+    }
+
+    @Override
+    public long getLSN() {
+        return LSN;
+    }
+
+    @Override
+    public void setLSN(long LSN) {
+        this.LSN = LSN;
+    }
+}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
deleted file mode 100644
index d46ade8..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogRecordHelper.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogManagerProperties;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-/**
- * An implementation of the @see ILogRecordHelper interface that provides API
- * for writing/reading of log header and checksum as well as validating log
- * record by checksum comparison. Every ILogManager implementation has an
- * associated ILogRecordHelper implementation.
- * == LogRecordFormat ==
- * [Header]
- * --------------------------- Header part1(17) : Both COMMIT and UPDATE log type have part1 fields
- * LogMagicNumber(4)
- * LogType(1)
- * JobId(4)
- * DatasetId(4) //stored in dataset_dataset in Metadata Node
- * PKHashValue(4)
- * --------------------------- Header part2(21) : Only UPDATE log type has part2 fields
- * PrevLSN(8) //only for UPDATE
- * ResourceId(8) //stored in .metadata of the corresponding index in NC node
- * ResourceMgrId(1)
- * LogRecordSize(4)
- * --------------------------- COMMIT doesn't have Body fields.
- * [Body] The Body size is given through the parameter reusableLogContentObjectLength
- * TupleFieldCount(4)
- * NewOp(1)
- * NewValueLength(4)
- * NewValue(NewValueLength)
- * OldOp(1)
- * OldValueLength(4)
- * OldValue(OldValueLength)
- * --------------------------- Both COMMIT and UPDATE have tail fields.
- * [Tail]
- * Checksum(8)
- */
-public class LogRecordHelper implements ILogRecordHelper {
-
-    private final int LOG_CHECKSUM_SIZE = 8;
-    private final int LOG_HEADER_PART1_SIZE = 17;
-    private final int LOG_HEADER_PART2_SIZE = 21;
-    private final int COMMIT_LOG_SIZE = LOG_HEADER_PART1_SIZE + LOG_CHECKSUM_SIZE;
-
-    private final int MAGIC_NO_POS = 0;
-    private final int LOG_TYPE_POS = 4;
-    private final int JOB_ID_POS = 5;
-    private final int DATASET_ID_POS = 9;
-    private final int PK_HASH_VALUE_POS = 13;
-    private final int PREV_LSN_POS = 17;
-    private final int RESOURCE_ID_POS = 25;
-    private final int RESOURCE_MGR_ID_POS = 33;
-    private final int LOG_RECORD_SIZE_POS = 34;
-
-    private ILogManager logManager;
-
-    public LogRecordHelper(ILogManager logManager) {
-        this.logManager = logManager;
-    }
-
-    @Override
-    public byte getLogType(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().getByte(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS);
-    }
-
-    @Override
-    public int getJobId(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS);
-    }
-
-    @Override
-    public int getDatasetId(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS);
-    }
-
-    @Override
-    public int getPKHashValue(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().readInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_POS);
-    }
-
-    @Override
-    public PhysicalLogLocator getPrevLSN(LogicalLogLocator logicalLogLocator) {
-        long prevLsnValue = (logicalLogLocator.getBuffer())
-                .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
-        PhysicalLogLocator previousLogLocator = new PhysicalLogLocator(prevLsnValue, logManager);
-        return previousLogLocator;
-    }
-
-    @Override
-    public boolean getPrevLSN(PhysicalLogLocator physicalLogLocator, LogicalLogLocator logicalLogLocator) {
-        long prevLsnValue = (logicalLogLocator.getBuffer())
-                .readLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS);
-        if (prevLsnValue == -1) {
-            return false;
-        }
-        physicalLogLocator.setLsn(prevLsnValue);
-        return true;
-    }
-
-    @Override
-    public long getResourceId(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getBuffer().readLong(logicalLogLocator.getMemoryOffset() + RESOURCE_ID_POS);
-    }
-
-    @Override
-    public byte getResourceMgrId(LogicalLogLocator logicalLogLocater) {
-        return logicalLogLocater.getBuffer().getByte(logicalLogLocater.getMemoryOffset() + RESOURCE_MGR_ID_POS);
-    }
-
-    @Override
-    public int getLogContentSize(LogicalLogLocator logicalLogLocater) {
-        if (getLogType(logicalLogLocater) == LogType.COMMIT || getLogType(logicalLogLocater) == LogType.ENTITY_COMMIT) {
-            return 0;
-        } else {
-            return logicalLogLocater.getBuffer().readInt(logicalLogLocater.getMemoryOffset() + LOG_RECORD_SIZE_POS);
-        }
-    }
-
-    @Override
-    public long getLogChecksum(LogicalLogLocator logicalLogLocator) {
-        return (logicalLogLocator.getBuffer()).readLong(logicalLogLocator.getMemoryOffset()
-                + getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator))
-                - LOG_CHECKSUM_SIZE);
-    }
-
-    @Override
-    public int getLogContentBeginPos(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getMemoryOffset() + getLogHeaderSize(getLogType(logicalLogLocator));
-    }
-
-    @Override
-    public int getLogContentEndPos(LogicalLogLocator logicalLogLocator) {
-        return logicalLogLocator.getMemoryOffset()
-                + getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator))
-                - LOG_CHECKSUM_SIZE;
-    }
-
-    @Override
-    public String getLogRecordForDisplay(LogicalLogLocator logicalLogLocator) {
-        StringBuilder builder = new StringBuilder();
-        byte logType = new Byte(getLogType(logicalLogLocator));
-        String logTypeDisplay = null;
-        switch (logType) {
-            case LogType.COMMIT:
-                logTypeDisplay = "COMMIT";
-                break;
-            case LogType.UPDATE:
-                logTypeDisplay = "UPDATE";
-                break;
-            case LogType.ENTITY_COMMIT:
-                logTypeDisplay = "ENTITY_COMMIT";
-                break;
-        }
-        builder.append(" LSN : ").append(logicalLogLocator.getLsn());
-        builder.append(" Log Type : ").append(logTypeDisplay);
-        builder.append(" Job Id : ").append(getJobId(logicalLogLocator));
-        builder.append(" Dataset Id : ").append(getDatasetId(logicalLogLocator));
-        builder.append(" PK Hash Value : ").append(getPKHashValue(logicalLogLocator));
-        if (logType == LogType.UPDATE) {
-            builder.append(" PrevLSN : ").append(getPrevLSN(logicalLogLocator).getLsn());
-            builder.append(" Resource Id : ").append(getResourceId(logicalLogLocator));
-            builder.append(" ResourceMgr Id : ").append(getResourceMgrId(logicalLogLocator));
-            builder.append(" Log Record Size : ").append(
-                    getLogRecordSize(logType, getLogContentSize(logicalLogLocator)));
-        }
-        return builder.toString();
-    }
-
-    @Override
-    public void writeLogHeader(LogicalLogLocator logicalLogLocator, byte logType, ITransactionContext context,
-            int datasetId, int PKHashValue, long prevLogicalLogLocator, long resourceId, byte resourceMgrId,
-            int logRecordSize) {
-
-        /* magic no */
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + MAGIC_NO_POS,
-                LogManagerProperties.LOG_MAGIC_NUMBER);
-
-        /* log type */
-        (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + LOG_TYPE_POS, logType);
-
-        /* jobId */
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + JOB_ID_POS, context.getJobId()
-                .getId());
-
-        /* datasetId */
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + DATASET_ID_POS, datasetId);
-
-        /* PK hash value */
-        (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + PK_HASH_VALUE_POS, PKHashValue);
-
-        if (logType == LogType.UPDATE) {
-            /* prevLSN */
-            (logicalLogLocator.getBuffer()).writeLong(logicalLogLocator.getMemoryOffset() + PREV_LSN_POS,
-                    prevLogicalLogLocator);
-
-            /* resourceId */
-            (logicalLogLocator.getBuffer())
-                    .writeLong(logicalLogLocator.getMemoryOffset() + RESOURCE_ID_POS, resourceId);
-
-            /* resourceMgr id */
-            (logicalLogLocator.getBuffer()).put(logicalLogLocator.getMemoryOffset() + RESOURCE_MGR_ID_POS,
-                    resourceMgrId);
-
-            /* log record size */
-            (logicalLogLocator.getBuffer()).writeInt(logicalLogLocator.getMemoryOffset() + LOG_RECORD_SIZE_POS,
-                    logRecordSize);
-
-        }
-    }
-
-    @Override
-    public boolean validateLogRecord(LogicalLogLocator logicalLogLocator) {
-        int logLength = this.getLogRecordSize(getLogType(logicalLogLocator), getLogContentSize(logicalLogLocator));
-        long expectedChecksum = DataUtil.getChecksum(logicalLogLocator.getBuffer(),
-                logicalLogLocator.getMemoryOffset(), logLength - LOG_CHECKSUM_SIZE);
-        long actualChecksum = getLogChecksum(logicalLogLocator);
-        return expectedChecksum == actualChecksum;
-    }
-
-    /**
-     * @param logType
-     * @param logBodySize
-     * @return
-     */
-    @Override
-    public int getLogRecordSize(byte logType, int logBodySize) {
-        if (logType == LogType.UPDATE) {
-            return LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE + LOG_CHECKSUM_SIZE + logBodySize;
-        } else {
-            return COMMIT_LOG_SIZE;
-        }
-    }
-
-    @Override
-    public int getLogHeaderSize(byte logType) {
-        if (logType == LogType.UPDATE) {
-            return LOG_HEADER_PART1_SIZE + LOG_HEADER_PART2_SIZE;
-        } else {
-            return LOG_HEADER_PART1_SIZE;
-        }
-    }
-
-    @Override
-    public int getLogChecksumSize() {
-        return LOG_CHECKSUM_SIZE;
-    }
-
-    public int getCommitLogSize() {
-        return COMMIT_LOG_SIZE;
-    }
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
index 99c0fef..3fc710a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/LogType.java
@@ -17,7 +17,7 @@
 public class LogType {
 
     public static final byte UPDATE = 0;
-    public static final byte COMMIT = 1;
+    public static final byte JOB_COMMIT = 1;
     public static final byte ENTITY_COMMIT = 2;
 
 }
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java
deleted file mode 100644
index d63ce1c..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/logging/MemBasedBuffer.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.logging;
-
-import java.nio.ByteBuffer;
-
-/**
- * Represents a memory-only buffer that is not associated with and physical
- * file.
- */
-public class MemBasedBuffer extends Buffer {
-
-    public MemBasedBuffer(byte[] content) {
-        super(ByteBuffer.wrap(content));
-    }
-
-    public MemBasedBuffer(ByteBuffer buffer) {
-        super(buffer);
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
index 110ddee..f2cb22a 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/CheckpointObject.java
@@ -20,24 +20,24 @@
 
     private static final long serialVersionUID = 1L;
  
-    private final long checkpointLSN;
-    private final long minMCTFirstLSN;
+    private final long checkpointLsn;
+    private final long minMCTFirstLsn;
     private final int maxJobId;
     private final long timeStamp;
 
-    public CheckpointObject(long checkpointLSN, long minMCTFirstLSN, int maxJobId, long timeStamp) {
-        this.checkpointLSN = checkpointLSN;
-        this.minMCTFirstLSN = minMCTFirstLSN;
+    public CheckpointObject(long checkpointLsn, long minMCTFirstLsn, int maxJobId, long timeStamp) {
+        this.checkpointLsn = checkpointLsn;
+        this.minMCTFirstLsn = minMCTFirstLsn;
         this.maxJobId = maxJobId;
         this.timeStamp = timeStamp;
     }
     
-    public long getCheckpointLSN() {
-        return checkpointLSN;
+    public long getCheckpointLsn() {
+        return checkpointLsn;
     }
 
-    public long getMinMCTFirstLSN() {
-        return minMCTFirstLSN;
+    public long getMinMCTFirstLsn() {
+        return minMCTFirstLsn;
     }
 
     public int getMaxJobId() {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
index a7d803e..1b07463 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/RecoveryManager.java
@@ -39,20 +39,12 @@
 import edu.uci.ics.asterix.common.api.ILocalResourceMetadata;
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
+import edu.uci.ics.asterix.common.ioopcallbacks.AbstractLSMIOOperationCallback;
 import edu.uci.ics.asterix.common.transactions.IAsterixAppRuntimeContextProvider;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
+import edu.uci.ics.asterix.common.transactions.ILogReader;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexResourceManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionManagementConstants;
@@ -60,23 +52,13 @@
 import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
-import edu.uci.ics.hyracks.storage.am.btree.impls.BTree;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndex;
 import edu.uci.ics.hyracks.storage.am.common.api.IIndexLifecycleManager;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndex;
-import edu.uci.ics.hyracks.storage.am.common.api.ITreeIndexMetaDataFrame;
 import edu.uci.ics.hyracks.storage.am.common.impls.NoOpOperationCallback;
-import edu.uci.ics.hyracks.storage.am.lsm.btree.impls.LSMBTreeDiskComponent;
-import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMComponent;
+import edu.uci.ics.hyracks.storage.am.common.ophelpers.IndexOperation;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndexAccessor;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.BlockingIOOperationCallbackWrapper;
-import edu.uci.ics.hyracks.storage.am.lsm.invertedindex.impls.LSMInvertedIndexDiskComponent;
-import edu.uci.ics.hyracks.storage.am.lsm.rtree.impls.LSMRTreeDiskComponent;
-import edu.uci.ics.hyracks.storage.am.rtree.impls.RTree;
-import edu.uci.ics.hyracks.storage.common.buffercache.IBufferCache;
-import edu.uci.ics.hyracks.storage.common.buffercache.ICachedPage;
-import edu.uci.ics.hyracks.storage.common.file.BufferedFileHandle;
 import edu.uci.ics.hyracks.storage.common.file.ILocalResourceRepository;
 import edu.uci.ics.hyracks.storage.common.file.LocalResource;
 
@@ -91,6 +73,7 @@
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(RecoveryManager.class.getName());
     private final TransactionSubsystem txnSubsystem;
+    private final LogManager logMgr;
     private final int checkpointHistory;
 
     /**
@@ -100,8 +83,9 @@
     private static final String CHECKPOINT_FILENAME_PREFIX = "checkpoint_";
     private SystemState state;
 
-    public RecoveryManager(TransactionSubsystem TransactionProvider) throws ACIDException {
-        this.txnSubsystem = TransactionProvider;
+    public RecoveryManager(TransactionSubsystem txnSubsystem) throws ACIDException {
+        this.txnSubsystem = txnSubsystem;
+        this.logMgr = (LogManager) txnSubsystem.getLogManager();
         this.checkpointHistory = this.txnSubsystem.getTransactionProperties().getCheckpointHistory();
     }
 
@@ -132,12 +116,15 @@
         //#. if minMCTFirstLSN is equal to -1 && 
         //   checkpointLSN in the checkpoint file is equal to the lastLSN in the log file,
         //   then return healthy state. Otherwise, return corrupted.
-        LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
-        if (checkpointObject.getMinMCTFirstLSN() == -1
-                && checkpointObject.getCheckpointLSN() == logMgr.getCurrentLsn().get()) {
+        if ((checkpointObject.getMinMCTFirstLsn() == -2 && logMgr.getAppendLSN() == 0)
+                || (checkpointObject.getMinMCTFirstLsn() == -1 && checkpointObject.getCheckpointLsn() == logMgr
+                        .getAppendLSN())) {
             state = SystemState.HEALTHY;
             return state;
         } else {
+            if (logMgr.getAppendLSN() == 0) {
+                throw new IllegalStateException("Transaction log files are lost.");
+            }
             state = SystemState.CORRUPTED;
             return state;
         }
@@ -151,146 +138,121 @@
 
         state = SystemState.RECOVERING;
 
-        ILogManager logManager = txnSubsystem.getLogManager();
-        ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
-
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("[RecoveryMgr] starting recovery ...");
         }
 
-        //winnerTxnTable is used to add pairs, <committed TxnId, the most recent commit LSN of the TxnId>
+        //winnerTxnTable is used to add pairs, <committed TxnId, the most recent commit Lsn of the TxnId>
         Map<TxnId, Long> winnerTxnTable = new HashMap<TxnId, Long>();
         TxnId tempKeyTxnId = new TxnId(-1, -1, -1);
-        byte logType;
 
         //#. read checkpoint file and set lowWaterMark where anaylsis and redo start
         CheckpointObject checkpointObject = readCheckpoint();
-        long lowWaterMarkLSN = checkpointObject.getMinMCTFirstLSN();
-        if (lowWaterMarkLSN == -1) {
-            lowWaterMarkLSN = 0;
+        long lowWaterMarkLsn = checkpointObject.getMinMCTFirstLsn();
+        if (lowWaterMarkLsn == -1 || lowWaterMarkLsn == -2) {
+            lowWaterMarkLsn = 0;
         }
         int maxJobId = checkpointObject.getMaxJobId();
-        int currentJobId;
 
         //-------------------------------------------------------------------------
         //  [ analysis phase ]
-        //  - collect all committed LSN 
+        //  - collect all committed Lsn 
         //  - if there are duplicate commits for the same TxnId, 
-        //    keep only the mostRecentCommitLSN among the duplicates.
+        //    keep only the mostRecentCommitLsn among the duplicates.
         //-------------------------------------------------------------------------
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("[RecoveryMgr] in analysis phase");
         }
 
-        //#. set log cursor to the lowWaterMarkLSN
-        ILogCursor logCursor = logManager.readLog(new PhysicalLogLocator(lowWaterMarkLSN, logManager),
-                new ILogFilter() {
-                    public boolean accept(IBuffer logs, long startOffset, int endOffset) {
-                        return true;
-                    }
-                });
-        LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-
-        //#. collect all committed txn's pairs,<TxnId, LSN>
-        while (logCursor.next(currentLogLocator)) {
-
-            if (LogManager.IS_DEBUG_MODE) {
-                System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+        //#. set log reader to the lowWaterMarkLsn
+        ILogReader logReader = logMgr.getLogReader(true);
+        logReader.initializeScan(lowWaterMarkLsn);
+        ILogRecord logRecord = logReader.next();
+        while (logRecord != null) {
+            if (IS_DEBUG_MODE) {
+                System.out.println(logRecord.getLogRecordForDisplay());
             }
-
-            logType = logRecordHelper.getLogType(currentLogLocator);
-
             //update max jobId
-            currentJobId = logRecordHelper.getJobId(currentLogLocator);
-            if (currentJobId > maxJobId) {
-                maxJobId = currentJobId;
+            if (logRecord.getJobId() > maxJobId) {
+                maxJobId = logRecord.getJobId();
             }
-
             TxnId commitTxnId = null;
-            switch (logType) {
+            switch (logRecord.getLogType()) {
                 case LogType.UPDATE:
                     if (IS_DEBUG_MODE) {
                         updateLogCount++;
                     }
                     break;
 
-                case LogType.COMMIT:
+                case LogType.JOB_COMMIT:
                 case LogType.ENTITY_COMMIT:
-                    commitTxnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
-                            logRecordHelper.getDatasetId(currentLogLocator),
-                            logRecordHelper.getPKHashValue(currentLogLocator));
-                    winnerTxnTable.put(commitTxnId, currentLogLocator.getLsn());
+                    commitTxnId = new TxnId(logRecord.getJobId(), logRecord.getDatasetId(), logRecord.getPKHashValue());
+                    winnerTxnTable.put(commitTxnId, logRecord.getLSN());
                     if (IS_DEBUG_MODE) {
                         commitLogCount++;
                     }
                     break;
 
                 default:
-                    throw new ACIDException("Unsupported LogType: " + logType);
+                    throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
             }
+            logRecord = logReader.next();
         }
 
         //-------------------------------------------------------------------------
         //  [ redo phase ]
         //  - redo if
-        //    1) The TxnId is committed --> gurantee durability
-        //      &&  
-        //    2) the currentLSN > maxDiskLastLSN of the index --> guarantee idempotance
+        //    1) The TxnId is committed && --> guarantee durability
+        //    2) lsn < commitLog's Lsn && --> deal with a case of pkHashValue collision
+        //    3) lsn > maxDiskLastLsn of the index --> guarantee idempotance
         //-------------------------------------------------------------------------
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("[RecoveryMgr] in redo phase");
         }
-        //#. set log cursor to the lowWaterMarkLSN again.
-        logCursor = logManager.readLog(new PhysicalLogLocator(lowWaterMarkLSN, logManager), new ILogFilter() {
-            public boolean accept(IBuffer logs, long startOffset, int endOffset) {
-                return true;
-            }
-        });
-        currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
+        //#. set log reader to the lowWaterMarkLsn again.
+        logReader.initializeScan(lowWaterMarkLsn);
 
         long resourceId;
-        byte resourceMgrId;
-        long maxDiskLastLSN;
-        long currentLSN = -1;
-        int resourceType;
+        long maxDiskLastLsn;
+        long lsn = -1;
+        long commitLsn = -1;
         ILSMIndex index = null;
         LocalResource localResource = null;
         ILocalResourceMetadata localResourceMetadata = null;
-        Map<Long, Long> resourceId2MaxLSNMap = new HashMap<Long, Long>();
-        List<ILSMComponent> immutableDiskIndexList = null;
+        Map<Long, Long> resourceId2MaxLsnMap = new HashMap<Long, Long>();
         TxnId jobLevelTxnId = new TxnId(-1, -1, -1);
-        boolean foundWinnerTxn;
+        boolean foundWinnerTxn = false;
 
         //#. get indexLifeCycleManager 
         IAsterixAppRuntimeContextProvider appRuntimeContext = txnSubsystem.getAsterixAppRuntimeContextProvider();
         IIndexLifecycleManager indexLifecycleManager = appRuntimeContext.getIndexLifecycleManager();
         ILocalResourceRepository localResourceRepository = appRuntimeContext.getLocalResourceRepository();
 
-        //#. redo
-        while (logCursor.next(currentLogLocator)) {
+        logRecord = logReader.next();
+        while (logRecord != null) {
+            lsn = logRecord.getLSN();
             foundWinnerTxn = false;
             if (LogManager.IS_DEBUG_MODE) {
-                System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+                System.out.println(logRecord.getLogRecordForDisplay());
             }
-
-            logType = logRecordHelper.getLogType(currentLogLocator);
-
-            switch (logType) {
+            switch (logRecord.getLogType()) {
                 case LogType.UPDATE:
-                    tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
-                            logRecordHelper.getDatasetId(currentLogLocator),
-                            logRecordHelper.getPKHashValue(currentLogLocator));
-                    jobLevelTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator), -1, -1);
+                    tempKeyTxnId.setTxnId(logRecord.getJobId(), logRecord.getDatasetId(), logRecord.getPKHashValue());
+                    jobLevelTxnId.setTxnId(logRecord.getJobId(), -1, -1);
                     if (winnerTxnTable.containsKey(tempKeyTxnId)) {
-                        currentLSN = winnerTxnTable.get(tempKeyTxnId);
-                        foundWinnerTxn = true;
+                        commitLsn = winnerTxnTable.get(tempKeyTxnId);
+                        if (lsn < commitLsn) {
+                            foundWinnerTxn = true;
+                        }
                     } else if (winnerTxnTable.containsKey(jobLevelTxnId)) {
-                        currentLSN = winnerTxnTable.get(jobLevelTxnId);
-                        foundWinnerTxn = true;
+                        commitLsn = winnerTxnTable.get(jobLevelTxnId);
+                        if (lsn < commitLsn) {
+                            foundWinnerTxn = true;
+                        }
                     }
 
                     if (foundWinnerTxn) {
-                        resourceId = logRecordHelper.getResourceId(currentLogLocator);
+                        resourceId = logRecord.getResourceId();
                         localResource = localResourceRepository.getResourceById(resourceId);
 
                         //get index instance from IndexLifeCycleManager
@@ -325,58 +287,20 @@
                             indexLifecycleManager.open(resourceId);
 
                             //#. get maxDiskLastLSN
-                            resourceType = localResource.getResourceType();
-                            immutableDiskIndexList = index.getImmutableComponents();
-
-                            maxDiskLastLSN = -1;
-                            switch (resourceType) {
-
-                                case ResourceType.LSM_BTREE:
-                                    for (ILSMComponent c : immutableDiskIndexList) {
-                                        BTree btree = ((LSMBTreeDiskComponent) c).getBTree();
-                                        maxDiskLastLSN = Math.max(getTreeIndexLSN(btree), maxDiskLastLSN);
-                                    }
-                                    break;
-
-                                case ResourceType.LSM_RTREE:
-                                    for (ILSMComponent c : immutableDiskIndexList) {
-                                        RTree rtree = ((LSMRTreeDiskComponent) c).getRTree();
-                                        maxDiskLastLSN = Math.max(getTreeIndexLSN(rtree), maxDiskLastLSN);
-                                    }
-                                    break;
-
-                                case ResourceType.LSM_INVERTED_INDEX:
-                                    for (ILSMComponent c : immutableDiskIndexList) {
-                                        BTree delKeyBtree = ((LSMInvertedIndexDiskComponent) c).getDeletedKeysBTree();
-                                        maxDiskLastLSN = Math.max(getTreeIndexLSN(delKeyBtree), maxDiskLastLSN);
-                                    }
-                                    break;
-
-                                default:
-                                    throw new ACIDException("Unsupported resouce type");
-                            }
+                            ILSMIndex lsmIndex = (ILSMIndex) index;
+                            BaseOperationTracker indexOpTracker = (BaseOperationTracker) lsmIndex.getOperationTracker();
+                            AbstractLSMIOOperationCallback abstractLSMIOCallback = (AbstractLSMIOOperationCallback) indexOpTracker
+                                    .getIOOperationCallback();
+                            maxDiskLastLsn = abstractLSMIOCallback.getComponentLSN(index.getImmutableComponents());
 
                             //#. set resourceId and maxDiskLastLSN to the map
-                            resourceId2MaxLSNMap.put(resourceId, maxDiskLastLSN);
+                            resourceId2MaxLsnMap.put(resourceId, maxDiskLastLsn);
                         } else {
-                            maxDiskLastLSN = resourceId2MaxLSNMap.get(resourceId);
+                            maxDiskLastLsn = resourceId2MaxLsnMap.get(resourceId);
                         }
 
-                        if (currentLSN > maxDiskLastLSN) {
-                            resourceMgrId = logRecordHelper.getResourceMgrId(currentLogLocator);
-
-                            // look up the repository to get the resource manager
-                            // register resourceMgr if it is not registered. 
-                            IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
-                                    .getTransactionalResourceMgr(resourceMgrId);
-                            if (resourceMgr == null) {
-                                resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
-                                txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
-                                        resourceMgrId, resourceMgr);
-                            }
-
-                            //redo finally.
-                            resourceMgr.redo(logRecordHelper, currentLogLocator);
+                        if (lsn > maxDiskLastLsn) {
+                            redo(logRecord);
                             if (IS_DEBUG_MODE) {
                                 redoCount++;
                             }
@@ -384,22 +308,26 @@
                     }
                     break;
 
-                case LogType.COMMIT:
+                case LogType.JOB_COMMIT:
                 case LogType.ENTITY_COMMIT:
                     //do nothing
                     break;
 
                 default:
-                    throw new ACIDException("Unsupported LogType: " + logType);
+                    throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
             }
+
+            logRecord = logReader.next();
         }
 
         //close all indexes
-        Set<Long> resourceIdList = resourceId2MaxLSNMap.keySet();
+        Set<Long> resourceIdList = resourceId2MaxLsnMap.keySet();
         for (long r : resourceIdList) {
             indexLifecycleManager.close(r);
         }
 
+        logReader.close();
+
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("[RecoveryMgr] recovery is completed.");
         }
@@ -409,33 +337,14 @@
         }
     }
 
-    //TODO
-    //This function came from the AbstractLSMIOOperationCallback class. 
-    //We'd better factor out this function into a component of reading/writing the local metadata of indexes.
-    private long getTreeIndexLSN(ITreeIndex treeIndex) throws HyracksDataException {
-        int fileId = treeIndex.getFileId();
-        IBufferCache bufferCache = treeIndex.getBufferCache();
-        ITreeIndexMetaDataFrame metadataFrame = treeIndex.getFreePageManager().getMetaDataFrameFactory().createFrame();
-        int metadataPageId = treeIndex.getFreePageManager().getFirstMetadataPage();
-        ICachedPage metadataPage = bufferCache.pin(BufferedFileHandle.getDiskPageId(fileId, metadataPageId), false);
-        metadataPage.acquireReadLatch();
-        try {
-            metadataFrame.setPage(metadataPage);
-            return metadataFrame.getLSN();
-        } finally {
-            metadataPage.releaseReadLatch();
-            bufferCache.unpin(metadataPage);
-        }
-    }
-
     @Override
     public synchronized void checkpoint(boolean isSharpCheckpoint) throws ACIDException, HyracksDataException {
 
+        long minMCTFirstLSN;
         if (isSharpCheckpoint && LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info("Starting sharp checkpoint ... ");
         }
 
-        LogManager logMgr = (LogManager) txnSubsystem.getLogManager();
         TransactionManager txnMgr = (TransactionManager) txnSubsystem.getTransactionManager();
         String logDir = logMgr.getLogManagerProperties().getLogDir();
 
@@ -446,10 +355,14 @@
         IIndexLifecycleManager indexLifecycleManager = txnSubsystem.getAsterixAppRuntimeContextProvider()
                 .getIndexLifecycleManager();
         List<IIndex> openIndexList = indexLifecycleManager.getOpenIndexes();
-        List<BlockingIOOperationCallbackWrapper> callbackList = new LinkedList<BlockingIOOperationCallbackWrapper>();
 
         //#. flush all in-memory components if it is the sharp checkpoint
         if (isSharpCheckpoint) {
+            ///////////////////////////////////////////////
+            //TODO : change the code inside the if statement into indexLifeCycleManager.flushAllDatasets()
+            //indexLifeCycleManager.flushAllDatasets();
+            ///////////////////////////////////////////////
+            List<BlockingIOOperationCallbackWrapper> callbackList = new LinkedList<BlockingIOOperationCallbackWrapper>();
             for (IIndex index : openIndexList) {
                 ILSMIndex lsmIndex = (ILSMIndex) index;
                 ILSMIndexAccessor indexAccessor = lsmIndex.createAccessor(NoOpOperationCallback.INSTANCE,
@@ -472,21 +385,20 @@
                     throw new ACIDException(e);
                 }
             }
-        }
-
-        //#. create and store the checkpointObject into the new checkpoint file
-        long minMCTFirstLSN = Long.MAX_VALUE;
-        long firstLSN;
-        if (openIndexList.size() > 0) {
-            for (IIndex index : openIndexList) {
-                firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
-                minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
-            }
+            minMCTFirstLSN = -2;
         } else {
-            minMCTFirstLSN = -1;
+            long firstLSN;
+            minMCTFirstLSN = Long.MAX_VALUE;
+            if (openIndexList.size() > 0) {
+                for (IIndex index : openIndexList) {
+                    firstLSN = ((BaseOperationTracker) ((ILSMIndex) index).getOperationTracker()).getFirstLSN();
+                    minMCTFirstLSN = Math.min(minMCTFirstLSN, firstLSN);
+                }
+            } else {
+                minMCTFirstLSN = -1;
+            }
         }
-
-        CheckpointObject checkpointObject = new CheckpointObject(logMgr.getCurrentLsn().get(), minMCTFirstLSN,
+        CheckpointObject checkpointObject = new CheckpointObject(logMgr.getAppendLSN(), minMCTFirstLSN,
                 txnMgr.getMaxJobId(), System.currentTimeMillis());
 
         FileOutputStream fos = null;
@@ -584,7 +496,7 @@
     }
 
     private File[] getPreviousCheckpointFiles() {
-        String logDir = txnSubsystem.getLogManager().getLogManagerProperties().getLogDir();
+        String logDir = ((LogManager) txnSubsystem.getLogManager()).getLogManagerProperties().getLogDir();
 
         File parentDir = new File(logDir);
 
@@ -620,8 +532,6 @@
      */
     @Override
     public void rollbackTransaction(ITransactionContext txnContext) throws ACIDException {
-        ILogManager logManager = txnSubsystem.getLogManager();
-        ILogRecordHelper logRecordHelper = logManager.getLogRecordHelper();
         Map<TxnId, List<Long>> loserTxnTable = new HashMap<TxnId, List<Long>>();
         TxnId tempKeyTxnId = new TxnId(-1, -1, -1);
 
@@ -629,15 +539,15 @@
         int commitLogCount = 0;
 
         // Obtain the first log record written by the Job
-        PhysicalLogLocator firstLSNLogLocator = txnContext.getFirstLogLocator();
-        PhysicalLogLocator lastLSNLogLocator = txnContext.getLastLogLocator();
+        long firstLSN = txnContext.getFirstLSN();
+        long lastLSN = txnContext.getLastLSN();
+        //TODO: make sure that the lastLsn is not updated anymore by another thread belonging to the same job.
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" rollbacking transaction log records from " + firstLSNLogLocator.getLsn() + " to "
-                    + lastLSNLogLocator.getLsn());
+            LOGGER.info(" rollbacking transaction log records from " + firstLSN + " to " + lastLSN);
         }
 
         // check if the transaction actually wrote some logs.
-        if (firstLSNLogLocator.getLsn() == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
+        if (firstLSN == TransactionManagementConstants.LogManagerConstants.TERMINAL_LSN) {
             if (LOGGER.isLoggable(Level.INFO)) {
                 LOGGER.info(" no need to roll back as there were no operations by the transaction "
                         + txnContext.getJobId());
@@ -645,72 +555,39 @@
             return;
         }
 
-        // While reading log records from firstLSN to lastLSN, collect uncommitted txn's LSNs 
-        ILogCursor logCursor;
-        try {
-            logCursor = logManager.readLog(firstLSNLogLocator, new ILogFilter() {
-                @Override
-                public boolean accept(IBuffer buffer, long startOffset, int length) {
-                    return true;
-                }
-            });
-        } catch (IOException e) {
-            throw new ACIDException("Failed to create LogCursor with LSN:" + firstLSNLogLocator.getLsn(), e);
-        }
-
-        LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-        boolean valid;
-        byte logType;
-        List<Long> undoLSNSet = null;
-
+        // While reading log records from firstLsn to lastLsn, collect uncommitted txn's Lsns
         if (LOGGER.isLoggable(Level.INFO)) {
-            LOGGER.info(" collecting loser transaction's LSNs from " + firstLSNLogLocator.getLsn() + " to "
-                    + +lastLSNLogLocator.getLsn());
+            LOGGER.info(" collecting loser transaction's LSNs from " + firstLSN + " to " + lastLSN);
         }
-
-        while (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
-            try {
-                valid = logCursor.next(currentLogLocator);
-            } catch (IOException e) {
-                throw new ACIDException("Failed to read log at LSN:" + currentLogLocator.getLsn(), e);
-            }
-            if (!valid) {
-                if (currentLogLocator.getLsn() != lastLSNLogLocator.getLsn()) {
-                    throw new ACIDException("LastLSN mismatch: " + lastLSNLogLocator.getLsn() + " vs "
-                            + currentLogLocator.getLsn() + " during Rollback a transaction( " + txnContext.getJobId()
-                            + ")");
-                } else {
-                    break;//End of Log File
-                }
-            }
-
+        boolean reachedLastLog = false;
+        List<Long> undoLSNSet = null;
+        ILogReader logReader = logMgr.getLogReader(false);
+        logReader.initializeScan(firstLSN);
+        ILogRecord logRecord = logReader.next();
+        while (logRecord != null) {
             if (IS_DEBUG_MODE) {
-                System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+                System.out.println(logRecord.getLogRecordForDisplay());
             }
 
-            tempKeyTxnId.setTxnId(logRecordHelper.getJobId(currentLogLocator),
-                    logRecordHelper.getDatasetId(currentLogLocator), logRecordHelper.getPKHashValue(currentLogLocator));
-            logType = logRecordHelper.getLogType(currentLogLocator);
-
-            switch (logType) {
+            tempKeyTxnId.setTxnId(logRecord.getJobId(), logRecord.getDatasetId(), logRecord.getPKHashValue());
+            switch (logRecord.getLogType()) {
                 case LogType.UPDATE:
                     undoLSNSet = loserTxnTable.get(tempKeyTxnId);
                     if (undoLSNSet == null) {
-                        TxnId txnId = new TxnId(logRecordHelper.getJobId(currentLogLocator),
-                                logRecordHelper.getDatasetId(currentLogLocator),
-                                logRecordHelper.getPKHashValue(currentLogLocator));
+                        TxnId txnId = new TxnId(logRecord.getJobId(), logRecord.getDatasetId(),
+                                logRecord.getPKHashValue());
                         undoLSNSet = new LinkedList<Long>();
                         loserTxnTable.put(txnId, undoLSNSet);
                     }
-                    undoLSNSet.add(currentLogLocator.getLsn());
+                    undoLSNSet.add(logRecord.getLSN());
                     if (IS_DEBUG_MODE) {
                         updateLogCount++;
-                        System.out.println("" + Thread.currentThread().getId() + "======> update["
-                                + currentLogLocator.getLsn() + "]:" + tempKeyTxnId);
+                        System.out.println("" + Thread.currentThread().getId() + "======> update[" + logRecord.getLSN()
+                                + "]:" + tempKeyTxnId);
                     }
                     break;
 
-                case LogType.COMMIT:
+                case LogType.JOB_COMMIT:
                 case LogType.ENTITY_COMMIT:
                     undoLSNSet = loserTxnTable.get(tempKeyTxnId);
                     if (undoLSNSet != null) {
@@ -718,14 +595,26 @@
                     }
                     if (IS_DEBUG_MODE) {
                         commitLogCount++;
-                        System.out.println("" + Thread.currentThread().getId() + "======> commit["
-                                + currentLogLocator.getLsn() + "]" + tempKeyTxnId);
+                        System.out.println("" + Thread.currentThread().getId() + "======> commit[" + logRecord.getLSN()
+                                + "]" + tempKeyTxnId);
                     }
                     break;
 
                 default:
-                    throw new ACIDException("Unsupported LogType: " + logType);
+                    throw new ACIDException("Unsupported LogType: " + logRecord.getLogType());
             }
+            if (logRecord.getLSN() == lastLSN) {
+                reachedLastLog = true;
+                break;
+            } else if (logRecord.getLSN() > lastLSN) {
+                throw new IllegalStateException("LastLSN mismatch");
+            }
+            logRecord = logReader.next();
+        }
+
+        if (!reachedLastLog) {
+            throw new ACIDException("LastLSN mismatch: " + lastLSN + " vs " + logRecord.getLSN()
+                    + " during Rollback a transaction( " + txnContext.getJobId() + ")");
         }
 
         //undo loserTxn's effect
@@ -735,7 +624,6 @@
 
         TxnId txnId = null;
         Iterator<Entry<TxnId, List<Long>>> iter = loserTxnTable.entrySet().iterator();
-        byte resourceMgrId;
         int undoCount = 0;
         while (iter.hasNext()) {
             //TODO 
@@ -750,36 +638,20 @@
                 // here, all the log records are UPDATE type. So, we don't need to check the type again.
 
                 //read the corresponding log record to be undone.
-                logManager.readLog(undoLSN, currentLogLocator);
-
+                logRecord = logReader.read(undoLSN);
+                assert logRecord != null;
                 if (IS_DEBUG_MODE) {
-                    System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(currentLogLocator));
+                    System.out.println(logRecord.getLogRecordForDisplay());
                 }
-
-                // extract the resource manager id from the log record.
-                resourceMgrId = logRecordHelper.getResourceMgrId(currentLogLocator);
-                if (LOGGER.isLoggable(Level.FINE)) {
-                    LOGGER.fine(logRecordHelper.getLogRecordForDisplay(currentLogLocator));
-                }
-
-                // look up the repository to get the resource manager
-                IResourceManager resourceMgr = txnSubsystem.getTransactionalResourceRepository()
-                        .getTransactionalResourceMgr(resourceMgrId);
-
-                // register resourceMgr if it is not registered. 
-                if (resourceMgr == null) {
-                    resourceMgr = new IndexResourceManager(resourceMgrId, txnSubsystem);
-                    txnSubsystem.getTransactionalResourceRepository().registerTransactionalResourceManager(
-                            resourceMgrId, resourceMgr);
-                }
-                resourceMgr.undo(logRecordHelper, currentLogLocator);
-
+                undo(logRecord);
                 if (IS_DEBUG_MODE) {
                     undoCount++;
                 }
             }
         }
 
+        logReader.close();
+
         if (LOGGER.isLoggable(Level.INFO)) {
             LOGGER.info(" undone loser transaction's effect");
         }
@@ -798,6 +670,53 @@
     public void stop(boolean dumpState, OutputStream os) {
         //no op
     }
+
+    private void undo(ILogRecord logRecord) {
+        try {
+            IIndex index = (IIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(logRecord.getResourceId());
+            ILSMIndexAccessor indexAccessor = (ILSMIndexAccessor) index.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            if (logRecord.getResourceType() == ResourceType.LSM_BTREE) {
+                if (logRecord.getOldOp() != IndexOperation.NOOP.ordinal()) {
+                    if (logRecord.getOldOp() == IndexOperation.DELETE.ordinal()) {
+                        indexAccessor.forceDelete(logRecord.getOldValue());
+                    } else {
+                        indexAccessor.forceInsert(logRecord.getOldValue());
+                    }
+                } else {
+                    indexAccessor.forcePhysicalDelete(logRecord.getNewValue());
+                }
+            } else {
+                if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+                    indexAccessor.forceInsert(logRecord.getNewValue());
+                } else {
+                    indexAccessor.forceDelete(logRecord.getNewValue());
+                }
+            }
+        } catch (Exception e) {
+            throw new IllegalStateException("Failed to undo", e);
+        }
+    }
+
+    private void redo(ILogRecord logRecord) {
+        try {
+            ILSMIndex index = (ILSMIndex) txnSubsystem.getAsterixAppRuntimeContextProvider().getIndexLifecycleManager()
+                    .getIndex(logRecord.getResourceId());
+            ILSMIndexAccessor indexAccessor = index.createAccessor(NoOpOperationCallback.INSTANCE,
+                    NoOpOperationCallback.INSTANCE);
+            if (logRecord.getNewOp() == IndexOperation.INSERT.ordinal()) {
+                indexAccessor.insert(logRecord.getNewValue());
+            } else if (logRecord.getNewOp() == IndexOperation.DELETE.ordinal()) {
+                indexAccessor.delete(logRecord.getNewValue());
+            } else {
+                throw new IllegalStateException("Unsupported OperationType: " + logRecord.getNewOp());
+            }
+            ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(logRecord.getLSN());
+        } catch (Exception e) {
+            throw new IllegalStateException("Failed to redo", e);
+        }
+    }
 }
 
 class TxnId {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
deleted file mode 100644
index e64ec01..0000000
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/recovery/TransactionTableEntry.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.service.recovery;
-
-import edu.uci.ics.asterix.common.transactions.ITransactionManager;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-
-/**
- * Represents a bookkeeping data-structure that is populated duing the analysis
- * phase of recovery. It contains for each transaction, the transaction state,
- * the LSN corresponding to the last log record written by the transaction and
- * the log record written by the transaction that needs to be undone.
- */
-public class TransactionTableEntry {
-
-    private long transactionId;
-    private ITransactionManager.TransactionState transactionState;
-    private PhysicalLogLocator lastLSN;
-    private PhysicalLogLocator undoNextLSN;
-
-    public TransactionTableEntry(long transactionId, ITransactionManager.TransactionState transactionState,
-            PhysicalLogLocator lastLSN, PhysicalLogLocator undoNextLSN) {
-        this.transactionId = transactionId;
-        this.transactionState = transactionState;
-        this.lastLSN = lastLSN;
-        this.undoNextLSN = undoNextLSN;
-    }
-
-    public long getTransactionId() {
-        return transactionId;
-    }
-
-    public void setTransactionId(long transactionId) {
-        this.transactionId = transactionId;
-    }
-
-    public ITransactionManager.TransactionState getTransactionState() {
-        return transactionState;
-    }
-
-    public void setTransactionState(ITransactionManager.TransactionState transactionState) {
-        this.transactionState = transactionState;
-    }
-
-    public PhysicalLogLocator getLastLSN() {
-        return lastLSN;
-    }
-
-    public void setLastLSN(PhysicalLogLocator lastLSN) {
-        this.lastLSN = lastLSN;
-    }
-
-    public PhysicalLogLocator getUndoNextLSN() {
-        return undoNextLSN;
-    }
-
-    public void setUndoNextLSN(PhysicalLogLocator undoNextLSN) {
-        this.undoNextLSN = undoNextLSN;
-    }
-
-}
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
index 432743a..2d94df6 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionContext.java
@@ -15,21 +15,20 @@
 package edu.uci.ics.asterix.transaction.management.service.transaction;
 
 import java.io.Serializable;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
 
 import edu.uci.ics.asterix.common.context.BaseOperationTracker;
+import edu.uci.ics.asterix.common.context.PrimaryIndexOperationTracker;
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.AbstractOperationCallback;
-import edu.uci.ics.asterix.common.transactions.ICloseable;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager.TransactionState;
 import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
+import edu.uci.ics.asterix.common.transactions.MutableLong;
+import edu.uci.ics.asterix.transaction.management.opcallbacks.PrimaryIndexModificationOperationCallback;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
-import edu.uci.ics.hyracks.storage.am.common.api.IModificationOperationCallback;
 import edu.uci.ics.hyracks.storage.am.lsm.common.api.ILSMIndex;
 import edu.uci.ics.hyracks.storage.am.lsm.common.impls.LSMOperationType;
 
@@ -44,73 +43,77 @@
 
     private static final long serialVersionUID = -6105616785783310111L;
     private TransactionSubsystem transactionSubsystem;
-    private LogicalLogLocator firstLogLocator;//firstLSN of the Job
-    private LogicalLogLocator lastLogLocator;//lastLSN of the Job
+    private final AtomicLong firstLSN;
+    private final AtomicLong lastLSN;
     private TransactionState txnState;
     private long startWaitTime;
     private int status;
-    private Set<ICloseable> resources = new HashSet<ICloseable>();
     private TransactionType transactionType = TransactionType.READ;
     private JobId jobId;
     private boolean exlusiveJobLevelCommit;
-
-    // List of indexes on which operations were performed on behalf of this transaction.
-    private final Set<ILSMIndex> indexes = new HashSet<ILSMIndex>();
-
-    // List of operation callbacks corresponding to the operand indexes. In particular, needed to track
-    // the number of active operations contributed by this transaction.
-    private final Set<AbstractOperationCallback> callbacks = new HashSet<AbstractOperationCallback>();
+    private final Map<MutableLong, BaseOperationTracker> indexMap;
+    private ILSMIndex primaryIndex;
+    private PrimaryIndexModificationOperationCallback primaryIndexCallback;
+    private PrimaryIndexOperationTracker primaryIndexOpTracker;
+    private final MutableLong tempResourceIdForRegister;
+    private final MutableLong tempResourceIdForSetLSN;
 
     public TransactionContext(JobId jobId, TransactionSubsystem transactionSubsystem) throws ACIDException {
         this.jobId = jobId;
         this.transactionSubsystem = transactionSubsystem;
-        init();
-    }
-
-    private void init() throws ACIDException {
-        firstLogLocator = LogUtil.getDummyLogicalLogLocator(transactionSubsystem.getLogManager());
-        lastLogLocator = LogUtil.getDummyLogicalLogLocator(transactionSubsystem.getLogManager());
+        firstLSN = new AtomicLong(-1);
+        lastLSN = new AtomicLong(-1);
         txnState = TransactionState.ACTIVE;
         startWaitTime = INVALID_TIME;
         status = ACTIVE_STATUS;
+        indexMap = new HashMap<MutableLong, BaseOperationTracker>();
+        primaryIndex = null;
+        tempResourceIdForRegister = new MutableLong();
+        tempResourceIdForSetLSN = new MutableLong();
     }
 
-    public void registerIndexAndCallback(ILSMIndex index, AbstractOperationCallback callback) {
-        synchronized (indexes) {
-            indexes.add(index);
-            callbacks.add(callback);
-        }
-    }
-
-    public void updateLastLSNForIndexes(long lastLSN) {
-        synchronized (indexes) {
-            for (ILSMIndex index : indexes) {
-                ((BaseOperationTracker) index.getOperationTracker()).updateLastLSN(lastLSN);
+    public void registerIndexAndCallback(long resourceId, ILSMIndex index, AbstractOperationCallback callback,
+            boolean isPrimaryIndex) {
+        synchronized (indexMap) {
+            if (isPrimaryIndex && primaryIndex == null) {
+                primaryIndex = index;
+                primaryIndexCallback = (PrimaryIndexModificationOperationCallback) callback;
+                primaryIndexOpTracker = (PrimaryIndexOperationTracker) index.getOperationTracker();
+            }
+            tempResourceIdForRegister.set(resourceId);
+            if (!indexMap.containsKey(tempResourceIdForRegister)) {
+                indexMap.put(new MutableLong(resourceId), ((BaseOperationTracker) index.getOperationTracker()));
             }
         }
     }
 
-    public void decreaseActiveTransactionCountOnIndexes() throws HyracksDataException {
-        synchronized (indexes) {
-            Set<BaseOperationTracker> opTrackers = new HashSet<BaseOperationTracker>();
-            Iterator<ILSMIndex> indexIt = indexes.iterator();
-            Iterator<AbstractOperationCallback> cbIt = callbacks.iterator();
-            while (indexIt.hasNext()) {
-                ILSMIndex index = indexIt.next();
-                opTrackers.add((BaseOperationTracker) index.getOperationTracker());
-                assert cbIt.hasNext();
+    //[Notice] 
+    //This method is called sequentially by the LogAppender threads. 
+    //However, the indexMap is concurrently read and modified through this method and registerIndexAndCallback()
+    //TODO: fix issues - 591, 609, 612, and 614.
+    @Override
+    public void setLastLSN(long resourceId, long LSN) {
+        synchronized (indexMap) {
+            firstLSN.compareAndSet(-1, LSN);
+            lastLSN.set(Math.max(lastLSN.get(), LSN));
+            tempResourceIdForSetLSN.set(resourceId);
+            //TODO; create version number tracker and keep LSNs there. 
+            BaseOperationTracker opTracker = indexMap.get(tempResourceIdForSetLSN);
+            opTracker.updateLastLSN(LSN);
+        }
+    }
+
+    @Override
+    public void notifyOptracker(boolean isJobLevelCommit) {
+        try {
+            if (isJobLevelCommit && exlusiveJobLevelCommit) {
+                primaryIndexOpTracker.exclusiveJobCommitted();
+            } else {
+                primaryIndexOpTracker
+                        .completeOperation(null, LSMOperationType.MODIFICATION, null, primaryIndexCallback);
             }
-            Iterator<BaseOperationTracker> trackerIt = opTrackers.iterator();
-            while (trackerIt.hasNext()) {
-                IModificationOperationCallback modificationCallback = (IModificationOperationCallback) cbIt.next();
-                BaseOperationTracker opTracker = (BaseOperationTracker) trackerIt.next();
-                if (exlusiveJobLevelCommit) {
-                    // For metadata transactions only
-                    opTracker.exclusiveJobCommitted();
-                } else {
-                    opTracker.completeOperation(null, LSMOperationType.MODIFICATION, null, modificationCallback);
-                }
-            }
+        } catch (HyracksDataException e) {
+            throw new IllegalStateException(e);
         }
     }
 
@@ -122,23 +125,21 @@
         return transactionType;
     }
 
-    public void addCloseableResource(ICloseable resource) {
-        resources.add(resource);
+    @Override
+    public long getFirstLSN() {
+        return firstLSN.get();
     }
 
-    public LogicalLogLocator getFirstLogLocator() {
-        return firstLogLocator;
+    @Override
+    public long getLastLSN() {
+        return lastLSN.get();
     }
 
-    public LogicalLogLocator getLastLogLocator() {
-        return lastLogLocator;
-    }
-
-    public void setLastLSN(long lsn) {
-        if (firstLogLocator.getLsn() == -1) {
-            firstLogLocator.setLsn(lsn);
+    public void setLastLSN(long LSN) {
+        if (firstLSN.get() == -1) {
+            firstLSN.set(LSN);
         }
-        lastLogLocator.setLsn(lsn);
+        lastLSN.set(LSN);
     }
 
     public JobId getJobId() {
@@ -169,12 +170,6 @@
         return txnState;
     }
 
-    public void releaseResources() throws ACIDException {
-        for (ICloseable closeable : resources) {
-            closeable.close(this);
-        }
-    }
-
     @Override
     public int hashCode() {
         return jobId.getId();
@@ -190,17 +185,12 @@
         exlusiveJobLevelCommit = true;
     }
 
-    @Override
-    public boolean isExlusiveJobLevelCommit() {
-        return exlusiveJobLevelCommit;
-    }
-
     public String prettyPrint() {
         StringBuilder sb = new StringBuilder();
         sb.append("\n" + jobId + "\n");
         sb.append("transactionType: " + transactionType);
-        sb.append("firstLogLocator: " + firstLogLocator.getLsn() + "\n");
-        sb.append("lastLogLocator: " + lastLogLocator.getLsn() + "\n");
+        sb.append("firstLSN: " + firstLSN.get() + "\n");
+        sb.append("lastLSN: " + lastLSN.get() + "\n");
         sb.append("TransactionState: " + txnState + "\n");
         sb.append("startWaitTime: " + startWaitTime + "\n");
         sb.append("status: " + status + "\n");
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
index 54ec8a1..cf614a5 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionManager.java
@@ -25,9 +25,11 @@
 
 import edu.uci.ics.asterix.common.exceptions.ACIDException;
 import edu.uci.ics.asterix.common.transactions.DatasetId;
+import edu.uci.ics.asterix.common.transactions.ILogRecord;
 import edu.uci.ics.asterix.common.transactions.ITransactionContext;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.JobId;
+import edu.uci.ics.asterix.transaction.management.service.logging.LogRecord;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
 import edu.uci.ics.hyracks.api.exceptions.HyracksDataException;
 import edu.uci.ics.hyracks.api.lifecycle.ILifeCycleComponent;
@@ -40,12 +42,14 @@
 
     public static final boolean IS_DEBUG_MODE = false;//true
     private static final Logger LOGGER = Logger.getLogger(TransactionManager.class.getName());
-    private final TransactionSubsystem transactionProvider;
+    private final TransactionSubsystem txnSubsystem;
     private Map<JobId, ITransactionContext> transactionContextRepository = new HashMap<JobId, ITransactionContext>();
     private AtomicInteger maxJobId = new AtomicInteger(0);
+    private final ILogRecord logRecord;
 
     public TransactionManager(TransactionSubsystem provider) {
-        this.transactionProvider = provider;
+        this.txnSubsystem = provider;
+        logRecord = new LogRecord();
     }
 
     @Override
@@ -57,7 +61,7 @@
             }
 
             try {
-                transactionProvider.getRecoveryManager().rollbackTransaction(txnContext);
+                txnSubsystem.getRecoveryManager().rollbackTransaction(txnContext);
             } catch (Exception ae) {
                 String msg = "Could not complete rollback! System is in an inconsistent state";
                 if (LOGGER.isLoggable(Level.SEVERE)) {
@@ -66,8 +70,7 @@
                 ae.printStackTrace();
                 throw new Error(msg);
             } finally {
-                txnContext.releaseResources();
-                transactionProvider.getLockManager().releaseLocks(txnContext);
+                txnSubsystem.getLockManager().releaseLocks(txnContext);
                 transactionContextRepository.remove(txnContext.getJobId());
                 txnContext.setTxnState(TransactionState.ABORTED);
             }
@@ -77,7 +80,7 @@
     @Override
     public ITransactionContext beginTransaction(JobId jobId) throws ACIDException {
         setMaxJobId(jobId.getId());
-        ITransactionContext txnContext = new TransactionContext(jobId, transactionProvider);
+        ITransactionContext txnContext = new TransactionContext(jobId, txnSubsystem);
         synchronized (this) {
             transactionContextRepository.put(jobId, txnContext);
         }
@@ -92,7 +95,7 @@
             ITransactionContext context = transactionContextRepository.get(jobId);
             if (context == null) {
                 context = transactionContextRepository.get(jobId);
-                context = new TransactionContext(jobId, transactionProvider);
+                context = new TransactionContext(jobId, txnSubsystem);
                 transactionContextRepository.put(jobId, context);
             }
             return context;
@@ -100,10 +103,9 @@
     }
 
     @Override
-    public void commitTransaction(ITransactionContext txnContext, DatasetId datasetId, int PKHashVal)
-            throws ACIDException {
-        synchronized (txnContext) {
-            if ((txnContext.getTxnState().equals(TransactionState.COMMITTED))) {
+    public void commitTransaction(ITransactionContext txnCtx, DatasetId datasetId, int PKHashVal) throws ACIDException {
+        synchronized (txnCtx) {
+            if ((txnCtx.getTxnState().equals(TransactionState.COMMITTED))) {
                 return;
             }
 
@@ -112,42 +114,25 @@
 
             //for entity-level commit
             if (PKHashVal != -1) {
-                boolean countIsZero = transactionProvider.getLockManager().unlock(datasetId, PKHashVal, txnContext,
-                        true);
-                if (!countIsZero) {
-                    // Lock count != 0 for a particular entity implies that the entity has been locked 
-                    // more than once (probably due to a hash collision in our current model).
-                    // It is safe to decrease the active transaction count on indexes since,  
-                    // by virtue of the counter not being zero, there is another transaction 
-                    // that has increased the transaction count. Thus, decreasing it will not 
-                    // allow the data to be flushed (yet). The flush will occur when the log page
-                    // flush thread decides to decrease the count for the last time.
-                    try {
-                        //decrease the transaction reference count on index
-                        txnContext.decreaseActiveTransactionCountOnIndexes();
-                    } catch (HyracksDataException e) {
-                        throw new ACIDException("failed to complete index operation", e);
-                    }
-                }
+                txnSubsystem.getLockManager().unlock(datasetId, PKHashVal, txnCtx, true);
                 return;
             }
 
             //for job-level commit
             try {
-                if (txnContext.getTransactionType().equals(ITransactionContext.TransactionType.READ_WRITE)) {
-                    transactionProvider.getLogManager().log(LogType.COMMIT, txnContext, -1, -1, -1, (byte) 0, 0, null,
-                            null, txnContext.getLastLogLocator());
+                if (txnCtx.getTransactionType().equals(ITransactionContext.TransactionType.READ_WRITE)) {
+                    logRecord.formCommitLogRecord(txnCtx, LogType.JOB_COMMIT, txnCtx.getJobId().getId(), -1, -1);
+                    txnSubsystem.getLogManager().log(logRecord);
                 }
-            } catch (ACIDException ae) {
+            } catch (Exception ae) {
                 if (LOGGER.isLoggable(Level.SEVERE)) {
-                    LOGGER.severe(" caused exception in commit !" + txnContext.getJobId());
+                    LOGGER.severe(" caused exception in commit !" + txnCtx.getJobId());
                 }
                 throw ae;
             } finally {
-                txnContext.releaseResources();
-                transactionProvider.getLockManager().releaseLocks(txnContext); // release
-                transactionContextRepository.remove(txnContext.getJobId());
-                txnContext.setTxnState(TransactionState.COMMITTED);
+                txnSubsystem.getLockManager().releaseLocks(txnCtx); // release
+                transactionContextRepository.remove(txnCtx.getJobId());
+                txnCtx.setTxnState(TransactionState.COMMITTED);
             }
         }
     }
@@ -164,7 +149,7 @@
 
     @Override
     public TransactionSubsystem getTransactionProvider() {
-        return transactionProvider;
+        return txnSubsystem;
     }
 
     public void setMaxJobId(int jobId) {
diff --git a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
index b76cf11..2dd7e64 100644
--- a/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
+++ b/asterix-transactions/src/main/java/edu/uci/ics/asterix/transaction/management/service/transaction/TransactionSubsystem.java
@@ -22,9 +22,7 @@
 import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionManager;
 import edu.uci.ics.asterix.common.transactions.ITransactionSubsystem;
-import edu.uci.ics.asterix.common.transactions.TransactionalResourceManagerRepository;
 import edu.uci.ics.asterix.transaction.management.service.locking.LockManager;
-import edu.uci.ics.asterix.transaction.management.service.logging.IndexLoggerRepository;
 import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
 import edu.uci.ics.asterix.transaction.management.service.recovery.CheckpointThread;
 import edu.uci.ics.asterix.transaction.management.service.recovery.RecoveryManager;
@@ -39,8 +37,6 @@
     private final ILockManager lockManager;
     private final ITransactionManager transactionManager;
     private final IRecoveryManager recoveryManager;
-    private final TransactionalResourceManagerRepository resourceRepository;
-    private final IndexLoggerRepository loggerRepository;
     private final IAsterixAppRuntimeContextProvider asterixAppRuntimeContextProvider;
     private final CheckpointThread checkpointThread;
     private final AsterixTransactionProperties txnProperties;
@@ -53,8 +49,6 @@
         this.logManager = new LogManager(this);
         this.lockManager = new LockManager(this);
         this.recoveryManager = new RecoveryManager(this);
-        this.loggerRepository = new IndexLoggerRepository(this);
-        this.resourceRepository = new TransactionalResourceManagerRepository();
         this.asterixAppRuntimeContextProvider = asterixAppRuntimeContextProvider;
         if (asterixAppRuntimeContextProvider != null) {
             this.checkpointThread = new CheckpointThread(recoveryManager,
@@ -81,14 +75,6 @@
         return recoveryManager;
     }
 
-    public TransactionalResourceManagerRepository getTransactionalResourceRepository() {
-        return resourceRepository;
-    }
-
-    public IndexLoggerRepository getTreeLoggerRepository() {
-        return loggerRepository;
-    }
-
     public IAsterixAppRuntimeContextProvider getAsterixAppRuntimeContextProvider() {
         return asterixAppRuntimeContextProvider;
     }
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
deleted file mode 100644
index 06379b4..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/BasicLogger.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.logging;
-
-import java.util.Random;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-
-public class BasicLogger implements ILogger {
-
-    private static long averageContentCreationTime = 0;
-    private static long count = 0;
-
-    public void log(ITransactionContext context, LogicalLogLocator wMemLSN, int length,
-            ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-
-        byte[] logContent = getRandomBytes(length);
-        try {
-            long startTime2 = System.nanoTime();
-
-            IBuffer buffer = (IBuffer) (wMemLSN.getBuffer());
-
-            /*
-             * synchronized(buffer){ buffer.position(wMemLSN.getMemoryOffset());
-             * buffer.put(logContent); }
-             */
-
-            byte[] logPageStorage = buffer.getArray();
-            System.arraycopy(logContent, 0, logPageStorage, wMemLSN.getMemoryOffset(), logContent.length);
-
-            /*
-             * for(int i=0;i<logContent.length;i++){
-             * ((IFileBasedBuffer)(wMemLSN.
-             * getBuffer())).put(wMemLSN.getMemoryOffset() + i, logContent[i]);
-             * }
-             */
-            long endTime2 = System.nanoTime();
-            averageContentCreationTime = ((averageContentCreationTime * count) + (endTime2 - startTime2)) / (++count);
-        } catch (Exception e) {
-            throw new ACIDException("", e);
-        }
-    }
-
-    public static long getAverageContentCreationTime() {
-        return averageContentCreationTime;
-    }
-
-    public static long getNumLogs() {
-        return averageContentCreationTime;
-    }
-
-    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    private static byte[] getRandomBytes(int size) {
-        byte[] b = new byte[size];
-        Random random = new Random();
-        int num = random.nextInt(30);
-        Integer number = (new Integer(num + 65));
-        byte numByte = number.byteValue();
-        for (int i = 0; i < size; i++) {
-            b[i] = numByte;
-        }
-        return b;
-    }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
deleted file mode 100644
index 8b9c970..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/IResource.java
+++ /dev/null
@@ -1,24 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.logging;
-
-import edu.uci.ics.asterix.common.transactions.ILogger;
-
-public interface IResource {
-
-    public byte[] getId();
-
-    public ILogger getLogger();
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
deleted file mode 100644
index e4a8e45..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/RecoverySimulator.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.logging.test;
-
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class RecoverySimulator {
-
-    private static IRecoveryManager recoveryManager;
-
-    public static void startRecovery() throws IOException, ACIDException {
-        recoveryManager.startRecovery(true);
-    }
-
-    public static void main(String args[]) throws IOException, ACIDException, AsterixException {
-        String id = "nc1";
-        try {
-            TransactionSubsystem factory = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
-                    new AsterixPropertiesAccessor()));
-            IRecoveryManager recoveryManager = factory.getRecoveryManager();
-            recoveryManager.startRecovery(true);
-        } catch (ACIDException acide) {
-            acide.printStackTrace();
-            throw acide;
-        }
-    }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
deleted file mode 100644
index a74b079..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/logging/test/TransactionWorkloadSimulator.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.logging.test;
-
-import java.util.Random;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.BasicLogger;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogActionType;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionContext;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class TransactionWorkloadSimulator {
-
-    public static ILogManager logManager;
-    public static ILockManager lockManager;
-    TransactionSubsystem provider;
-
-    public static WorkloadProperties workload;
-    Transaction[] transactions;
-
-    public TransactionWorkloadSimulator(WorkloadProperties workload) {
-        this.workload = workload;
-        transactions = new Transaction[workload.numActiveThreads];
-    }
-
-    public void beginWorkload() throws ACIDException, AsterixException {
-        provider = new TransactionSubsystem("nc1", null, new AsterixTransactionProperties(
-                new AsterixPropertiesAccessor()));
-        logManager = provider.getLogManager();
-        lockManager = provider.getLockManager();
-        provider.getTransactionalResourceRepository().registerTransactionalResourceManager(DummyResourceMgr.id,
-                new DummyResourceMgr());
-        Transaction[] transactions = new Transaction[workload.numActiveThreads];
-        long startTime = System.nanoTime();
-        for (int i = 0; i < workload.numActiveThreads; i++) {
-            transactions[i] = new Transaction(provider, "Transaction " + (i + 1), workload.singleTransaction);
-            transactions[i].start();
-        }
-        for (int i = 0; i < workload.numActiveThreads; i++) {
-            try {
-                transactions[i].join();
-            } catch (InterruptedException ignore) {
-            }
-        }
-
-        for (int i = 0; i < workload.numActiveThreads; i++) {
-            provider.getTransactionManager().commitTransaction(transactions[i].getContext(), new DatasetId(-1), -1);
-        }
-
-        long endTime = System.nanoTime();
-        int totalLogs = Transaction.logCount.get();
-        System.out.println(" Total logs :" + totalLogs);
-        long timeTaken = ((endTime - startTime) / 1000000);
-        System.out.println(" total time :" + timeTaken);
-        System.out.println(" throughput :" + totalLogs * 1000 / timeTaken + " logs/sec");
-        long totalBytesWritten = Transaction.logByteCount.get();
-        System.out.println(" bytes written :" + totalBytesWritten);
-        System.out.println(" IO throughput " + totalBytesWritten * 1000 / timeTaken + " bytes/sec");
-        System.out.println(" Avg Content Creation time :" + BasicLogger.getAverageContentCreationTime());
-    }
-
-    public static void main(String args[]) throws AsterixException {
-        WorkloadProperties workload = new WorkloadProperties();
-        TransactionWorkloadSimulator simulator = new TransactionWorkloadSimulator(workload);
-        try {
-            simulator.beginWorkload();
-        } catch (ACIDException acide) {
-            acide.printStackTrace();
-        }
-
-    }
-}
-
-class SingleTransactionContextFactory {
-    private static TransactionContext context;
-
-    public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
-        if (context == null) {
-            context = new TransactionContext(JobIdFactory.generateJobId(), provider);
-        }
-        return context;
-    }
-}
-
-class MultipleTransactionContextFactory {
-
-    public static TransactionContext getContext(TransactionSubsystem provider) throws ACIDException {
-        return new TransactionContext(JobIdFactory.generateJobId(), provider);
-    }
-}
-
-class Transaction extends Thread {
-
-    public static AtomicInteger logCount = new AtomicInteger(0);
-    public static AtomicLong logByteCount = new AtomicLong(0);
-    Random random = new Random();
-    BasicLogger logger = new BasicLogger();
-    LogicalLogLocator memLSN;
-    String name;
-    TransactionContext context;
-    //private byte[] resourceID = new byte[1];
-    private int resourceID;
-    private int myLogCount = 0;
-    private TransactionSubsystem transactionProvider;
-    private ILogManager logManager;
-    private DatasetId tempDatasetId = new DatasetId(-1);
-
-    public Transaction(TransactionSubsystem provider, String name, boolean singleTransaction) throws ACIDException {
-        this.name = name;
-        this.transactionProvider = provider;
-        if (singleTransaction) {
-            context = SingleTransactionContextFactory.getContext(transactionProvider);
-        } else {
-            context = MultipleTransactionContextFactory.getContext(transactionProvider);
-        }
-        memLSN = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
-        logManager = transactionProvider.getLogManager();
-    }
-
-    public TransactionContext getContext() {
-        return context;
-    }
-
-    @Override
-    public void run() {
-        if (TransactionWorkloadSimulator.workload.minLogsPerTransactionThread == TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread) {
-            TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread++;
-        }
-        int numLogs = TransactionWorkloadSimulator.workload.minLogsPerTransactionThread
-                + random.nextInt(TransactionWorkloadSimulator.workload.maxLogsPerTransactionThread
-                        - TransactionWorkloadSimulator.workload.minLogsPerTransactionThread);
-        int total = 0;
-        LogicalLogLocator memLSN = LogUtil.getDummyLogicalLogLocator(logManager);
-        if (TransactionWorkloadSimulator.workload.maxLogSize == TransactionWorkloadSimulator.workload.minLogSize) {
-            TransactionWorkloadSimulator.workload.maxLogSize++;
-        }
-        if (TransactionWorkloadSimulator.workload.singleResource) {
-            int choice = random.nextInt(2);
-            resourceID = (byte) (choice % 2);
-        } else {
-            random.nextInt(resourceID);
-        }
-        boolean retry = false;
-        byte lockMode = -1;
-        try {
-            for (int i = 0; i < numLogs - 1; i++) {
-                int logSize = TransactionWorkloadSimulator.workload.minLogSize
-                        + random.nextInt(TransactionWorkloadSimulator.workload.maxLogSize
-                                - TransactionWorkloadSimulator.workload.minLogSize);
-                total += logSize;
-
-                byte logType = LogType.UPDATE;
-                byte logActionType = LogActionType.REDO_UNDO;
-                long pageId = 0;
-                if (!retry) {
-                    lockMode = (byte) (random.nextInt(2));
-                }
-                tempDatasetId.setId(resourceID);
-                TransactionWorkloadSimulator.lockManager.lock(tempDatasetId, -1, lockMode, context);
-                TransactionWorkloadSimulator.logManager.log(logType, context, resourceID, -1, resourceID,
-                        ResourceType.LSM_BTREE, logSize, null, logger, memLSN);
-                retry = false;
-                Thread.currentThread().sleep(TransactionWorkloadSimulator.workload.thinkTime);
-                logCount.incrementAndGet();
-                logByteCount.addAndGet(logSize
-                        + TransactionWorkloadSimulator.logManager.getLogRecordHelper().getLogHeaderSize(logType)
-                        + TransactionWorkloadSimulator.logManager.getLogRecordHelper().getLogChecksumSize());
-                myLogCount++;
-            }
-        } catch (ACIDException acide) {
-            acide.printStackTrace();
-        } catch (Exception ie) {
-            ie.printStackTrace();
-        }
-    }
-
-}
-
-class WorkloadProperties {
-    public int numActiveThreads = 200;
-    public long thinkTime = 0; // (in mesecs)
-    public int minLogsPerTransactionThread = 5;
-    public int maxLogsPerTransactionThread = 5;
-    public int minLogSize = 1024 - 51;
-    public int maxLogSize = 1024 - 51;
-    public float commitFraction = 0.5f;
-    public float rollbackFraction = 0.1f;
-    public boolean singleTransaction = false;
-    public boolean singleResource = true;
-}
-
-class ResourceMgrInfo {
-    public static final byte BTreeResourceMgrId = 1;
-    public static final byte MetadataResourceMgrId = 2;
-}
-
-class DummyResourceMgr implements IResourceManager {
-
-    public static final byte id = 1;
-
-    @Override
-    public void redo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void undo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public byte getResourceManagerId() {
-        // TODO Auto-generated method stub
-        return 1;
-    }
-
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
deleted file mode 100644
index 2eea43e..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileLogger.java
+++ /dev/null
@@ -1,66 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.test;
-
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.ReusableLogContentObject;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileLogger implements ILogger {
-
-    IResource resource;
-    String logRecordContent;
-
-    public FileLogger(IResource resource) {
-        this.resource = resource;
-    }
-
-    public int generateLogRecordContent(int currentValue, int finalValue) {
-        StringBuilder builder = new StringBuilder();
-        builder.append("" + currentValue + " " + finalValue);
-        logRecordContent = new String(builder);
-        return resource.getId().length + logRecordContent.length();
-    }
-
-    @Override
-    public void preLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-    @Override
-    public void log(ITransactionContext context, final LogicalLogLocator memLSN, int logContentSize,
-            ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        byte[] buffer = memLSN.getBuffer().getArray();
-        byte[] content = logRecordContent.getBytes();
-        for (int i = 0; i < resource.getId().length; i++) {
-            buffer[memLSN.getMemoryOffset() + i] = resource.getId()[i];
-        }
-        for (int i = 0; i < content.length; i++) {
-            buffer[memLSN.getMemoryOffset() + resource.getId().length + i] = content[i];
-        }
-    }
-
-    @Override
-    public void postLog(ITransactionContext context, ReusableLogContentObject reusableLogContentObject) throws ACIDException {
-        // TODO Auto-generated method stub
-
-    }
-
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
deleted file mode 100644
index 072738f..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResource.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.test;
-
-import java.io.BufferedReader;
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileReader;
-import java.io.FileWriter;
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.transactions.FileUtil;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileResource implements IResource {
-
-    private byte[] resourceId = new byte[] { 1 };
-    private ILogger logger;
-
-    private File file;
-    private int memCounter = 0;
-    private int diskCounter = 0;
-
-    public int getMemoryCounter() {
-        return memCounter;
-    }
-
-    public int getDiskCounter() {
-        return diskCounter;
-    }
-
-    public static enum CounterOperation {
-        INCREMENT,
-        DECREMENT,
-    };
-
-    public FileResource(String fileDir, String fileName) throws IOException {
-        File dirFile = new File(fileDir);
-        if (!dirFile.exists()) {
-            FileUtil.createNewDirectory(fileDir);
-        }
-        file = new File(fileDir + "/" + fileName);
-        if (!file.exists()) {
-            FileUtil.createFileIfNotExists(file.getAbsolutePath());
-            BufferedWriter writer = new BufferedWriter(new FileWriter(file));
-            writer.write("0");
-            writer.flush();
-        } else {
-            FileReader fileReader = new FileReader(file);
-            BufferedReader bufferedReader = new BufferedReader(fileReader);
-            String content = bufferedReader.readLine();
-            diskCounter = Integer.parseInt(content);
-        }
-        logger = new FileLogger(this);
-    }
-
-    public synchronized void increment() {
-        memCounter++;
-    }
-
-    public synchronized void decrement() {
-        memCounter--;
-    }
-
-    public synchronized void setValue(int value) {
-        memCounter = value;
-    }
-
-    public synchronized void sync() throws IOException {
-        BufferedWriter writer = new BufferedWriter(new FileWriter(file));
-        writer.write("" + memCounter);
-        writer.flush();
-    }
-
-    public synchronized boolean checkIfValueInSync(int expectedValue) throws IOException {
-        FileReader fileReader = new FileReader(file);
-        BufferedReader bufferedReader = new BufferedReader(fileReader);
-        String content = bufferedReader.readLine();
-        return content.equals("" + expectedValue);
-    }
-
-    @Override
-    public byte[] getId() {
-        return resourceId;
-    }
-
-    @Override
-    public ILogger getLogger() {
-        return logger;
-    }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
deleted file mode 100644
index 5a6e408..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/FileResourceManager.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.test;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-
-class FileResourceManager implements IResourceManager {
-
-    public static final byte id = 100;
-    private Map<String, FileResource> transactionalResources = new HashMap<String, FileResource>();
-
-    public void registerTransactionalResource(IResource resource) throws ACIDException {
-        if (resource instanceof FileResource) {
-            if (transactionalResources.get(new String(resource.getId())) == null) {
-                transactionalResources.put(new String(resource.getId()), (FileResource) resource);
-            }
-        } else {
-            throw new ACIDException(" invalid resource type :" + resource);
-        }
-    }
-
-    @Override
-    public byte getResourceManagerId() {
-        return id;
-    }
-
-    @Override
-    public void undo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException {
-        LogRecordInfo logRecordInfo = new LogRecordInfo(logRecordHelper, memLSN);
-        FileResource fileManager = transactionalResources.get(logRecordInfo.getResourceId());
-        if (fileManager == null) {
-            throw new ACIDException(" Un-registered transactional resource :" + logRecordInfo.getResourceId());
-        }
-        fileManager.setValue(logRecordInfo.getBeforeValue());
-    }
-
-    @Override
-    public void redo(ILogRecordHelper logRecordHelper, LogicalLogLocator memLSN) throws ACIDException {
-        LogRecordInfo logRecordInfo = new LogRecordInfo(logRecordHelper, memLSN);
-        FileResource fileManager = transactionalResources.get(new String(logRecordInfo.getResourceId()));
-        if (fileManager == null) {
-            throw new ACIDException(" Un-registered transactional resource :" + logRecordInfo.getResourceId());
-        }
-        fileManager.setValue(logRecordInfo.getAfterValue());
-    }
-
-}
-
-class LogRecordInfo {
-
-    byte[] resourceId;
-    int beforeValue;
-    int afterValue;
-
-    public LogRecordInfo(ILogRecordHelper logParser, LogicalLogLocator memLSN) throws ACIDException {
-        int logContentBeginPos = logParser.getLogContentBeginPos(memLSN);
-        int logContentEndPos = logParser.getLogContentEndPos(memLSN);
-        byte[] bufferContent = memLSN.getBuffer().getArray();
-        resourceId = new byte[] { bufferContent[logContentBeginPos] };
-        String content = new String(bufferContent, logContentBeginPos + resourceId.length, logContentEndPos
-                - (logContentBeginPos + resourceId.length));
-        beforeValue = Integer.parseInt(content.split(" ")[0]);
-        afterValue = Integer.parseInt(content.split(" ")[1]);
-    }
-
-    public byte[] getResourceId() {
-        return resourceId;
-    }
-
-    public void setResourceId(byte[] resourceId) {
-        this.resourceId = resourceId;
-    }
-
-    public int getAfterValue() {
-        return afterValue;
-    }
-
-    public void setFinalValue(int afterValue) {
-        this.afterValue = afterValue;
-    }
-
-    public int getBeforeValue() {
-        return beforeValue;
-    }
-
-    public void setBeforeValue(int beforeValue) {
-        this.beforeValue = beforeValue;
-    }
-
-}
\ No newline at end of file
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
deleted file mode 100644
index 8e13356..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/LogRecordReader.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.test;
-
-import java.io.IOException;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.transactions.IBuffer;
-import edu.uci.ics.asterix.common.transactions.ILogCursor;
-import edu.uci.ics.asterix.common.transactions.ILogFilter;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogRecordHelper;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.common.transactions.PhysicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogManager;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class LogRecordReader {
-
-    ILogManager logManager;
-
-    public LogRecordReader(TransactionSubsystem factory) throws ACIDException {
-        logManager = factory.getLogManager();
-    }
-
-    public LogRecordReader(ILogManager logManager) {
-        this.logManager = logManager;
-    }
-
-    public void readLogs(long startingLsn) throws IOException, ACIDException {
-        ILogRecordHelper parser = logManager.getLogRecordHelper();
-        PhysicalLogLocator lsn = new PhysicalLogLocator(startingLsn, logManager);
-        ILogCursor logCursor = logManager.readLog(lsn, new ILogFilter() {
-            @Override
-            public boolean accept(IBuffer buffer, long startOffset, int length) {
-                return true;
-            }
-        });
-        LogicalLogLocator currentLogLocator = LogUtil.getDummyLogicalLogLocator(logManager);
-        int logCount = 0;
-        while (true) {
-            boolean logValidity = logCursor.next(currentLogLocator);
-            if (logValidity) {
-                System.out.println(++logCount + parser.getLogRecordForDisplay(currentLogLocator));
-            } else {
-                break;
-            }
-        }
-    }
-
-    public void readLogRecord(long lsnValue) throws IOException, ACIDException {
-        LogicalLogLocator memLSN = null;
-        logManager.readLog(lsnValue, memLSN);
-        System.out.println(logManager.getLogRecordHelper().getLogRecordForDisplay(memLSN));
-    }
-
-    /**
-     * @param args
-     */
-    public static void main(String[] args) throws ACIDException, Exception {
-        LogManager logManager = new LogManager(null, "nc1");
-        LogRecordReader logReader = new LogRecordReader(logManager);
-        logReader.readLogs(0);
-    }
-
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
deleted file mode 100644
index 5195658..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionRecoverySimulator.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.test;
-
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-
-public class TransactionRecoverySimulator {
-
-    public boolean recoverSystem() throws ACIDException {
-        return true;
-    }
-
-    public static void main(String args[]) {
-        TransactionRecoverySimulator recoverySimulator = new TransactionRecoverySimulator();
-    }
-}
diff --git a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java b/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
deleted file mode 100644
index 4bebbc4..0000000
--- a/asterix-transactions/src/test/java/edu/uci/ics/asterix/transaction/management/test/TransactionSimulator.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * Copyright 2009-2013 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.transaction.management.test;
-
-import java.io.IOException;
-import java.util.Random;
-
-import edu.uci.ics.asterix.common.config.AsterixPropertiesAccessor;
-import edu.uci.ics.asterix.common.config.AsterixTransactionProperties;
-import edu.uci.ics.asterix.common.exceptions.ACIDException;
-import edu.uci.ics.asterix.common.exceptions.AsterixException;
-import edu.uci.ics.asterix.common.transactions.DatasetId;
-import edu.uci.ics.asterix.common.transactions.ILockManager;
-import edu.uci.ics.asterix.common.transactions.ILogManager;
-import edu.uci.ics.asterix.common.transactions.ILogger;
-import edu.uci.ics.asterix.common.transactions.IRecoveryManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager;
-import edu.uci.ics.asterix.common.transactions.IResourceManager.ResourceType;
-import edu.uci.ics.asterix.common.transactions.ITransactionContext;
-import edu.uci.ics.asterix.common.transactions.ITransactionManager;
-import edu.uci.ics.asterix.common.transactions.JobId;
-import edu.uci.ics.asterix.common.transactions.LogUtil;
-import edu.uci.ics.asterix.common.transactions.LogicalLogLocator;
-import edu.uci.ics.asterix.transaction.management.logging.IResource;
-import edu.uci.ics.asterix.transaction.management.service.logging.LogType;
-import edu.uci.ics.asterix.transaction.management.service.transaction.JobIdFactory;
-import edu.uci.ics.asterix.transaction.management.service.transaction.TransactionSubsystem;
-
-public class TransactionSimulator {
-
-    private ITransactionManager transactionManager;
-    private ILogManager logManager;
-    private ILockManager lockManager;
-    private IRecoveryManager recoveryManager;
-    private IResourceManager resourceMgr;
-    private ILogger logger;
-    private IResource resource;
-    private LogicalLogLocator memLSN;
-    private TransactionSubsystem transactionProvider;
-
-    public TransactionSimulator(IResource resource, IResourceManager resourceMgr) throws ACIDException, AsterixException {
-        String id = "nc1";
-        transactionProvider = new TransactionSubsystem(id, null, new AsterixTransactionProperties(
-                new AsterixPropertiesAccessor()));
-        transactionManager = transactionProvider.getTransactionManager();
-        logManager = transactionProvider.getLogManager();
-        lockManager = transactionProvider.getLockManager();
-        recoveryManager = transactionProvider.getRecoveryManager();
-        transactionProvider.getTransactionalResourceRepository().registerTransactionalResourceManager(
-                resourceMgr.getResourceManagerId(), resourceMgr);
-        this.resourceMgr = resourceMgr;
-        this.logger = resource.getLogger();
-        this.resource = resource;
-        memLSN = LogUtil.getDummyLogicalLogLocator(transactionProvider.getLogManager());
-    }
-
-    public ITransactionContext beginTransaction() throws ACIDException {
-        JobId jobId = JobIdFactory.generateJobId();
-        return transactionManager.beginTransaction(jobId);
-    }
-
-    public void executeTransactionOperation(ITransactionContext txnContext, FileResource.CounterOperation operation)
-            throws ACIDException {
-        // lockManager.lock(txnContext, resourceId, 0);
-        ILogManager logManager = transactionProvider.getLogManager();
-        int currentValue = ((FileResource) resource).getMemoryCounter();
-        int finalValue;
-        switch (operation) {
-            case INCREMENT:
-                finalValue = currentValue + 1;
-                int logRecordLength = ((FileLogger) logger).generateLogRecordContent(currentValue, finalValue);
-                logManager.log(LogType.UPDATE, txnContext, 1, -1, 1, ResourceType.LSM_BTREE, 0, null, logger, memLSN);
-                ((FileResource) resource).increment();
-                break;
-            case DECREMENT:
-                finalValue = currentValue - 1;
-                logRecordLength = ((FileLogger) logger).generateLogRecordContent(currentValue, finalValue);
-                logManager.log(LogType.UPDATE, txnContext, 1, -1, 1, ResourceType.LSM_BTREE, 0, null, logger, memLSN);
-                ((FileResource) resource).decrement();
-                break;
-        }
-
-    }
-
-    public void commitTransaction(ITransactionContext context) throws ACIDException {
-        transactionManager.commitTransaction(context, new DatasetId(-1), -1);
-    }
-
-    public void recover() throws ACIDException, IOException {
-        recoveryManager.startRecovery(true);
-        ((FileResource) resource).sync();
-    }
-
-    /**
-     * @param args
-     */
-    public static void main(String[] args) throws IOException, ACIDException, AsterixException {
-        String fileDir = "testdata";
-        String fileName = "counterFile";
-        IResource resource = new FileResource(fileDir, fileName);
-        FileResourceManager resourceMgr = new FileResourceManager();
-        resourceMgr.registerTransactionalResource(resource);
-        int existingValue = ((FileResource) resource).getDiskCounter();
-
-        TransactionSimulator txnSimulator = new TransactionSimulator(((FileResource) resource), resourceMgr);
-        int numTransactions = 2;
-        Schedule schedule = new Schedule(numTransactions);
-
-        for (int i = 0; i < numTransactions; i++) {
-            ITransactionContext context = txnSimulator.beginTransaction();
-            txnSimulator.executeTransactionOperation(context, schedule.getOperations()[i]);
-            if (schedule.getWillCommit()[i]) {
-                txnSimulator.commitTransaction(context);
-            }
-        }
-
-        int finalExpectedValue = existingValue + schedule.getDeltaChange();
-        txnSimulator.recover();
-        boolean isCorrect = ((FileResource) resource).checkIfValueInSync(finalExpectedValue);
-        System.out.println(" Did recovery happen correctly " + isCorrect);
-    }
-
-}
-
-class ResourceMgrIds {
-
-    public static final byte FileResourceMgrId = 1;
-
-}
-
-class Schedule {
-
-    private int numCommittedIncrements;
-    private int numCommittedDecrements;
-
-    private FileResource.CounterOperation[] operations;
-    private Boolean[] willCommit;
-
-    public Boolean[] getWillCommit() {
-        return willCommit;
-    }
-
-    private Random random = new Random();
-
-    public int getDeltaChange() {
-        return numCommittedIncrements - numCommittedDecrements;
-    }
-
-    public Schedule(int numTransactions) {
-        operations = new FileResource.CounterOperation[numTransactions];
-        willCommit = new Boolean[numTransactions];
-        for (int i = 0; i < numTransactions; i++) {
-            willCommit[i] = random.nextBoolean();
-            int nextOp = random.nextInt(2);
-            FileResource.CounterOperation op = nextOp == 0 ? FileResource.CounterOperation.INCREMENT
-                    : FileResource.CounterOperation.DECREMENT;
-            operations[i] = op;
-            if (willCommit[i]) {
-                if (op.equals(FileResource.CounterOperation.INCREMENT)) {
-                    numCommittedIncrements++;
-                } else {
-                    numCommittedDecrements++;
-                }
-            }
-        }
-
-    }
-
-    public String toString() {
-        StringBuilder builder = new StringBuilder();
-        for (int i = 0; i < operations.length; i++) {
-            builder.append(" operation " + operations[i]);
-            if (willCommit[i]) {
-                builder.append(" commit ");
-            } else {
-                builder.append(" abort ");
-            }
-        }
-
-        builder.append(" number of committed increments " + numCommittedIncrements);
-        builder.append(" number of committed decrements " + numCommittedDecrements);
-        return new String(builder);
-    }
-
-    public FileResource.CounterOperation[] getOperations() {
-        return operations;
-    }
-
-}