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;
- }
-
-}