[teiid-commits] teiid SVN: r4609 - in branches/7.7.x: engine/src/main/java/org/teiid/common/buffer/impl and 3 other directories.

teiid-commits at lists.jboss.org teiid-commits at lists.jboss.org
Mon Nov 4 14:45:56 EST 2013


Author: jolee
Date: 2013-11-04 14:45:55 -0500 (Mon, 04 Nov 2013)
New Revision: 4609

Added:
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/OutOfDiskException.java
Modified:
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/Cache.java
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/ExtensibleBufferedOutputStream.java
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/FileStorageManager.java
   branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/MemoryStorageManager.java
   branches/7.7.x/engine/src/main/resources/org/teiid/query/i18n.properties
   branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java
   branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java
   branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestFileStorageManager.java
   branches/7.7.x/test-integration/perf/src/test/java/org/teiid/query/eval/TestEnginePerformance.java
Log:
TEIID-2714: Buffer defrag not working

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/Cache.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/Cache.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/Cache.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -92,5 +92,7 @@
 	 * @param id
 	 */
 	boolean remove(Long gid, Long id);
+
+	void shutdown();
 	
 }
\ No newline at end of file

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -59,4 +59,15 @@
 	protected int flushDirect(int i) throws IOException {
 		return i;
 	}
+
+	public void writeLong(long v) throws IOException {
+		write((byte)(v >>> 56));
+		write((byte)(v >>> 48));
+		write((byte)(v >>> 40));
+		write((byte)(v >>> 32));
+		write((byte)(v >>> 24));
+		write((byte)(v >>> 16));
+		write((byte)(v >>> 8));
+		write((byte)(v >>> 0));
+	}
 }
\ No newline at end of file

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockStore.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -79,6 +79,12 @@
 			//TODO: there is still an extra buffer being created here, we could FileChannels to do better
 			byte[] b = new byte[BufferFrontedFileStoreCache.BLOCK_SIZE];
 			int read = 0;
+			long newLength = blockOffset+blockSize;
+			if (fs.getLength() < newLength) {
+				//grow by whole blocks
+				//TODO: could pad the growth
+				fs.setLength(newLength); 
+			}
 			while ((read = is.read(b, 0, b.length)) != -1) {
 				fs.write(blockOffset, b, 0, read);
 				blockOffset+=read;

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferFrontedFileStoreCache.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -22,7 +22,6 @@
 
 package org.teiid.common.buffer.impl;
 
-import java.io.DataInputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.ObjectInput;
@@ -43,6 +42,7 @@
 import java.util.concurrent.Semaphore;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
@@ -101,8 +101,10 @@
  */
 public class BufferFrontedFileStoreCache implements Cache<PhysicalInfo>, StorageManager {
 	
+	private static final int FULL_DEFRAG_TRUNCATE_TIMEOUT = 10000;
+	private static final long TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(120);
 	private static final int DEFAULT_MIN_DEFRAG = 1 << 26;
-	private static final byte[] HEADER_SKIP_BUFFER = new byte[16];
+	private static final int HEADER_BYTES = 16;
 	private static final int EVICTION_SCANS = 2;
 
 	public static final int DEFAuLT_MAX_OBJECT_SIZE = 1 << 23;
@@ -373,106 +375,157 @@
 
 	private ExecutorService asynchPool = ExecutorUtils.newFixedThreadPool(2, "FileStore Worker"); //$NON-NLS-1$
 	private AtomicBoolean defragRunning = new AtomicBoolean();
+	private AtomicInteger freedCounter = new AtomicInteger();
+	
+	private int truncateInterval = 10;
 	//defrag to release freespace held by storage files
-	private final Runnable defragTask = new Runnable() {
+	final class DefragTask implements Runnable {
+		private AtomicInteger runs = new AtomicInteger();
 		
 		@Override
 		public void run() {
+			int count = runs.incrementAndGet();
 			try {
-				if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
-					LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Running defrag"); //$NON-NLS-1$ 
+				defrag(false);
+				if ((count%truncateInterval)==0) {
+					truncate(false);
 				}
-				for (int i = 0; i < sizeBasedStores.length; i++) {
-					BlockStore blockStore = sizeBasedStores[i];
-					for (int segment = 0; segment < blockStore.stores.length; segment++) {
-						if (!shouldDefrag(blockStore, segment)) {
-							continue;
-						}
-						try {
-							boolean sleep = false;
-							do {
-								if (sleep) {
-									Thread.sleep(100); //let the file activity quite down
-								}
-								sleep = true;
-								int relativeBlockToMove = blockStore.blocksInUse.compactHighestBitSet(segment);
-								if (!shouldDefrag(blockStore, segment)) {
-									truncate(blockStore, segment);
-									break;
-								}
-								//move the block if possible
-								InputStream is = blockStore.stores[segment].createInputStream(relativeBlockToMove * blockStore.blockSize, blockStore.blockSize);
-								DataInputStream dis = new DataInputStream(is);
-								Long gid = null;
-								Long oid = null;
-								try {
-									gid = dis.readLong();
-									oid = dis.readLong();
-								} catch (IOException e) {
-									continue; //can happen the bit was set and no data exists
-								}
-								dis.reset(); //move back to the beginning
-								Map<Long, PhysicalInfo> map = physicalMapping.get(gid);
-								if (map == null) {
+			} catch (Throwable t) {
+				LogManager.logWarning(LogConstants.CTX_BUFFER_MGR, t, "Uncaught exception durring defrag");
+			} finally {
+				defragRunning.set(false);
+			}
+		}
+		
+		private long truncate(boolean anySpace) {
+			long freed = 0;
+			for (int i = 0; i < sizeBasedStores.length; i++) {
+				BlockStore blockStore = sizeBasedStores[i];
+				for (int segment = 0; segment < blockStore.stores.length; segment++) {
+					freed += truncate(blockStore, segment, anySpace);
+				}
+			}
+			if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
+				LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Finished truncate reclaimed", freed); //$NON-NLS-1$ 
+			}
+			return freed;
+		}
+
+		private void defrag(boolean all) {
+			if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
+				LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Running defrag"); //$NON-NLS-1$ 
+			}
+			for (int i = 0; i < sizeBasedStores.length; i++) {
+				BlockStore blockStore = sizeBasedStores[i];
+				for (int segment = 0; segment < blockStore.stores.length; segment++) {
+					if (!shouldDefrag(blockStore, segment, all)) {
+						continue;
+					}
+					if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
+						LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Defraging store", i, "segment", segment, "length", blockStore.stores[segment].getLength()); //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ 
+					}
+					try {
+						for (int retries = 0; retries < 10; retries++) {
+							int relativeBlockToMove = blockStore.blocksInUse.compactHighestBitSet(segment);
+							if (!shouldDefrag(blockStore, segment, all) || relativeBlockToMove == -1) {
+								break;
+							}
+							//move the block if possible
+							InputStream is = blockStore.stores[segment].createInputStream(relativeBlockToMove * blockStore.blockSize, blockStore.blockSize);
+							Long gid = null;
+							Long oid = null;
+							try {
+								gid = readLong(is);
+								oid = readLong(is);
+							} catch (IOException e) {
+								continue; //can happen the bit was set and no data exists
+							}
+							is.reset(); //move back to the beginning
+							Map<Long, PhysicalInfo> map = physicalMapping.get(gid);
+							if (map == null) {
+								continue;
+							}
+							PhysicalInfo info = map.get(oid);
+							if (info == null) {
+								continue;
+							}
+							int bitIndex = relativeBlockToMove + (segment * blockStore.blocksInUse.getBitsPerSegment());
+							synchronized (info) {
+								info.await(true, false);
+								if (info.block == EMPTY_ADDRESS) {
 									continue;
 								}
-								PhysicalInfo info = map.get(oid);
-								if (info == null) {
+								if (info.block != bitIndex) {
+									//we've marked a bit in use, but haven't yet written new data
 									continue;
 								}
-								int bitIndex = relativeBlockToMove + (segment * blockStore.blocksInUse.getBitsPerSegment());
-								synchronized (info) {
-									info.await(true, false);
-									if (info.block == EMPTY_ADDRESS) {
-										continue;
+							}
+							int newBlock = blockStore.writeToStorageBlock(info, is);
+							synchronized (info) {
+								info.await(true, true);
+								if (info.block == EMPTY_ADDRESS) {
+									//already removed;
+									if (newBlock != EMPTY_ADDRESS) {
+										blockStore.blocksInUse.clear(newBlock);
 									}
-									if (info.block != bitIndex) {
-										//we've marked a bit in use, but haven't yet written new data
-										continue;
-									}
+									continue;
 								}
-								int newBlock = blockStore.writeToStorageBlock(info, dis);
-								synchronized (info) {
-									info.await(true, true);
-									if (info.block == EMPTY_ADDRESS) {
-										//already removed;
-										if (newBlock != EMPTY_ADDRESS) {
-											blockStore.blocksInUse.clear(newBlock);
-										}
-										continue;
-									}
-									info.block = newBlock;
-									blockStore.blocksInUse.clear(bitIndex);
-								}
-								sleep = false;
-							} while (shouldDefrag(blockStore, segment));
-						} catch (IOException e) {
-							LogManager.logWarning(LogConstants.CTX_BUFFER_MGR, e, "Error performing defrag"); //$NON-NLS-1$
-						} catch (InterruptedException e) {
-							throw new TeiidRuntimeException(e);
+								info.block = newBlock;
+								blockStore.blocksInUse.clear(bitIndex);
+							}
 						}
+					} catch (IOException e) {
+						LogManager.logWarning(LogConstants.CTX_BUFFER_MGR, e, "Error performing defrag");
 					}
 				}
-			} finally {
-				defragRunning.set(false);
 			}
 		}
 
-		private void truncate(BlockStore blockStore, int segment) {
+		private long readLong(InputStream is) throws IOException {
+			long val = 0;
+			for (int k = 0; k < 8; k++) {
+				val += ((is.read() & 255) << (56-k*8));
+			}
+			return val;
+		}
+
+		private long truncate(BlockStore blockStore, int segment, boolean anySpace) {
 			//truncate the file
 			blockStore.locks[segment].writeLock().lock();
 			try {
-				int endBlock = blockStore.blocksInUse.getHighestBitSet(segment);
-				long newLength = (endBlock + 1) * blockStore.blockSize; 
-				blockStore.stores[segment].setLength(newLength);
+				int endBlock = blockStore.blocksInUse.compactHighestBitSet(segment);
+				long newLength = (endBlock + 1) * blockStore.blockSize;
+				long oldLength = blockStore.stores[segment].getLength();
+				if (anySpace) {
+					if (newLength < oldLength) {
+						blockStore.stores[segment].setLength(newLength);
+						if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
+							LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Truncating segment", segment, "to", newLength); //$NON-NLS-1$ //$NON-NLS-2$ 
+						}
+						return oldLength - newLength;
+					}
+				} else {
+					long desiredLength = ((oldLength/blockStore.blockSize)/2)*blockStore.blockSize;
+					if (newLength < oldLength && newLength <= desiredLength && oldLength - desiredLength >= 2*minDefrag) {
+						blockStore.stores[segment].setLength(desiredLength);
+						if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
+							LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Truncating segment", segment, "to", desiredLength); //$NON-NLS-1$ //$NON-NLS-2$ 
+						}
+					}
+					return oldLength - desiredLength;
+				}
 			} catch (IOException e) {
 				LogManager.logWarning(LogConstants.CTX_BUFFER_MGR, e, "Error performing defrag truncate"); //$NON-NLS-1$
 			} finally {
 				blockStore.locks[segment].writeLock().unlock();
 			}
+			return 0;
 		}
 	};
-	private AtomicBoolean cleanerRunning = new AtomicBoolean();
+	final DefragTask defragTask = new DefragTask();
+	private long lastFullRun;
+	
+	AtomicBoolean cleanerRunning = new AtomicBoolean();
 	private final Runnable cleaningTask = new Runnable() {
 		
 		@Override
@@ -594,9 +647,9 @@
 			hasPermit = true;
 			blockManager = getBlockManager(s.getId(), entry.getId(), EMPTY_ADDRESS);
 			BlockOutputStream bos = new BlockOutputStream(blockManager, memoryBlocks);
+			bos.writeLong(s.getId());
+			bos.writeLong(entry.getId());
 			ObjectOutput dos = new ObjectOutputStream(bos);
-			dos.writeLong(s.getId());
-			dos.writeLong(entry.getId());
 			dos.writeInt(entry.getSizeEstimate());
             s.serialize(entry.getObject(), dos);
             dos.close();
@@ -722,8 +775,10 @@
 			if (lock != null) {
 				is = readIntoMemory(info, is, lock, memoryBlocks);
 			}
+			for (int i = 0; i < HEADER_BYTES; i++) {
+				is.read();
+			}
 			ObjectInput dis = new ObjectInputStream(is);
-			dis.readFully(HEADER_SKIP_BUFFER);
 			int sizeEstimate = dis.readInt();
 			CacheEntry ce = new CacheEntry(new CacheKey(oid, 1, 1), sizeEstimate, serializer.deserialize(dis), ref, true);
 			return ce;
@@ -904,7 +959,31 @@
 				storageWrites.getAndIncrement();
 				BlockInputStream is = new BlockInputStream(bm, memoryBlockCount); 
 				BlockStore blockStore = sizeBasedStores[sizeIndex];
-				block = blockStore.writeToStorageBlock(info, is);
+				for (int i = 0; i < 3; i++) {
+					try {
+						block = blockStore.writeToStorageBlock(info, is);
+						break;
+					} catch (OutOfDiskException e) {
+						switch (i) {
+						case 0:
+							//the first attempt is to trim the existing files
+							defragTask.truncate(true);
+							break;
+						case 1:
+							synchronized (this) {
+								if (System.currentTimeMillis() - lastFullRun > FULL_DEFRAG_TRUNCATE_TIMEOUT) {
+									defragTask.defrag(true);
+									defragTask.truncate(true);
+									lastFullRun = System.currentTimeMillis();
+								}
+							}
+							break;
+						case 2:
+							//give up, there isn't enough memory available
+							throw e;
+						}
+					}
+				}
 			}
 		} catch (IOException e) {
 			if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
@@ -938,9 +1017,8 @@
 						if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL)) {
 							LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Freed storage data block", info.block, "of size", blockStore.blockSize); //$NON-NLS-1$ //$NON-NLS-2$
 						}
-						int segment = info.block/blockStore.blocksInUse.getBitsPerSegment();
 						if (!defragRunning.get() 
-								&& shouldDefrag(blockStore, segment) 
+								&& (freedCounter.getAndIncrement()&Short.MAX_VALUE)==Short.MAX_VALUE //should be several gigabytes of turn over
 								&& defragRunning.compareAndSet(false, true)) {
 							this.asynchPool.execute(defragTask);
 						}
@@ -965,7 +1043,7 @@
 		return result;
 	}
 
-	boolean shouldDefrag(BlockStore blockStore, int segment) {
+	boolean shouldDefrag(BlockStore blockStore, int segment, boolean all) {
 		int highestBitSet = blockStore.blocksInUse.getHighestBitSet(segment);
 		int bitsSet = blockStore.blocksInUse.getBitsSet(segment);
 		highestBitSet = Math.max(bitsSet, Math.max(0, highestBitSet));
@@ -973,7 +1051,7 @@
 			return false;
 		}
 		int freeBlocks = highestBitSet-bitsSet;
-		return freeBlocks > (highestBitSet>>2) && freeBlocks*blockStore.blockSize > minDefrag;
+		return freeBlocks > (highestBitSet>>(all?3:1)) && freeBlocks*blockStore.blockSize > minDefrag;
 	}
 
 	/**
@@ -1097,5 +1175,25 @@
 	public void setBufferManager(BufferManagerImpl bufferManager) {
 		this.bufferManager = bufferManager;
 	}
+	
+	public void setTruncateInterval(int truncateInterval) {
+		this.truncateInterval = truncateInterval;
+	}
 
+	public long getDiskUsage() {
+		long result = 0;
+		for (int i = 0; i < sizeBasedStores.length; i++) {
+			BlockStore blockStore = sizeBasedStores[i];
+			for (int segment = 0; segment < blockStore.stores.length; segment++) {
+				result += blockStore.stores[segment].getLength();
+			}
+		}
+		return result;
+	}
+
+	@Override
+	public void shutdown() {
+		this.asynchPool.shutdownNow();
+	}
+	
 }
\ No newline at end of file

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -128,6 +128,8 @@
 		}
 	}
 
+	private Cleaner cleaner;
+	
 	/**
 	 * This estimate is based upon adding the value to 2/3 maps and having CacheEntry/PhysicalInfo keys
 	 */
@@ -387,7 +389,8 @@
 	private static final Timer timer = new Timer("BufferManager Cleaner", true); //$NON-NLS-1$
 	
 	public BufferManagerImpl() {
-		timer.schedule(new Cleaner(this), 15000, 15000);
+		this.cleaner = new Cleaner(this);
+		timer.schedule(cleaner, 15000, 15000);
 	}
 	
 	void clearSoftReference(BatchSoftReference bsr) {
@@ -829,6 +832,7 @@
 	
 	AtomicInteger removed = new AtomicInteger();
 	
+	
 	CacheEntry remove(Long gid, Long batch, boolean prefersMemory) {
 		if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR, MessageLevel.TRACE)) {
 			LogManager.logTrace(LogConstants.CTX_BUFFER_MGR, "Removing batch from BufferManager", batch); //$NON-NLS-1$
@@ -940,6 +944,12 @@
 	}
 
 	public void shutdown() {
+		this.cache.shutdown();
+		this.cache = null;
+		this.memoryEntries.clear();
+		this.evictionQueue.getEvictionQueue().clear();
+		//this.initialEvictionQueue.getEvictionQueue().clear();
+		this.cleaner.cancel();
 	}
 
 	@Override

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/ConcurrentBitSet.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -237,8 +237,8 @@
 		int highestBitSet = 0;
 		synchronized (s) {
 			highestBitSet = s.highestBitSet;
-			if (highestBitSet <= 0) {
-				return 0;
+			if (highestBitSet < 0) {
+				return -1;
 			}
 			if (s.bitSet.get(highestBitSet)) {
 				return highestBitSet;

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/ExtensibleBufferedOutputStream.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/ExtensibleBufferedOutputStream.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/ExtensibleBufferedOutputStream.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -39,6 +39,11 @@
     	ensureBuffer();
 		buf.put((byte)b);
     }
+    
+    public void write(byte b) throws IOException {
+    	ensureBuffer();
+		buf.put(b);
+    }
 
 	private void ensureBuffer() throws IOException {
 		if (buf != null) {

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/FileStorageManager.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/FileStorageManager.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/FileStorageManager.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -35,6 +35,7 @@
 import org.teiid.common.buffer.FileStore;
 import org.teiid.common.buffer.StorageManager;
 import org.teiid.core.TeiidComponentException;
+import org.teiid.logging.LogConstants;
 import org.teiid.logging.LogManager;
 import org.teiid.logging.MessageLevel;
 import org.teiid.query.QueryPlugin;
@@ -54,6 +55,8 @@
 	private AtomicLong usedBufferSpace = new AtomicLong();
 	private AtomicInteger fileCounter = new AtomicInteger();
 	
+	private AtomicLong sample = new AtomicLong();
+	
 	private class FileInfo {
     	private File file;
         private RandomAccessFile fileData;       // may be null if not open
@@ -156,17 +159,18 @@
 				//this is a weak check, concurrent access may push us over the max.  we are just trying to prevent large overage allocations
 				long used = usedBufferSpace.get() + bytesUsed;
 				if (used > maxBufferSpace) {
-					//TODO: trigger a compaction before this is thrown
-					throw new IOException(QueryPlugin.Util.getString("FileStoreageManager.space_exhausted", maxBufferSpace)); //$NON-NLS-1$
+					throw new OutOfDiskException(QueryPlugin.Util.getString("FileStoreageManager.space_exhausted", bytesUsed, used, maxBufferSpace)); //$NON-NLS-1$
 				}
 			}
 			fileAccess.setLength(newLength);
 			long used = usedBufferSpace.addAndGet(bytesUsed);
+			if (LogManager.isMessageToBeRecorded(org.teiid.logging.LogConstants.CTX_BUFFER_MGR, MessageLevel.DETAIL) && (sample.getAndIncrement() % 100) == 0) {
+				LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "sampling bytes used:", used); //$NON-NLS-1$
+			}
 			if (bytesUsed > 0 && used > maxBufferSpace) {
 				fileAccess.setLength(currentLength);
 				usedBufferSpace.addAndGet(-bytesUsed);
-				//TODO: trigger a compaction before this is thrown
-				throw new IOException(QueryPlugin.Util.getString("FileStoreageManager.space_exhausted", maxBufferSpace)); //$NON-NLS-1$
+				throw new OutOfDiskException(QueryPlugin.Util.getString("FileStoreageManager.space_exhausted", bytesUsed, used, maxBufferSpace)); //$NON-NLS-1$
 			}
 		}
 	    

Modified: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/MemoryStorageManager.java
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/MemoryStorageManager.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/MemoryStorageManager.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -179,5 +179,10 @@
 			return new ArrayList<Long>(group.keySet());
 		}
 	}
+
+	@Override
+	public void shutdown() {
+		
+	}
 	
 }
\ No newline at end of file

Copied: branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/OutOfDiskException.java (from rev 4608, branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/BlockOutputStream.java)
===================================================================
--- branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/OutOfDiskException.java	                        (rev 0)
+++ branches/7.7.x/engine/src/main/java/org/teiid/common/buffer/impl/OutOfDiskException.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -0,0 +1,35 @@
+/*
+ * JBoss, Home of Professional Open Source.
+ * See the COPYRIGHT.txt file distributed with this work for information
+ * regarding copyright ownership.  Some portions may be licensed
+ * to Red Hat, Inc. under one or more contributor license agreements.
+ * 
+ * This library is free software; you can redistribute it and/or
+ * modify it under the terms of the GNU Lesser General Public
+ * License as published by the Free Software Foundation; either
+ * version 2.1 of the License, or (at your option) any later version.
+ * 
+ * This library is distributed in the hope that it will be useful,
+ * but WITHOUT ANY WARRANTY; without even the implied warranty of
+ * MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE.  See the GNU
+ * Lesser General Public License for more details.
+ * 
+ * You should have received a copy of the GNU Lesser General Public
+ * License along with this library; if not, write to the Free Software
+ * Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA
+ * 02110-1301 USA.
+ */
+
+package org.teiid.common.buffer.impl;
+
+import java.io.IOException;
+
+public class OutOfDiskException extends IOException {
+	
+	private static final long serialVersionUID = -1332091322315668612L;
+
+	public OutOfDiskException(String message) {
+		super(message);
+	}
+
+}

Modified: branches/7.7.x/engine/src/main/resources/org/teiid/query/i18n.properties
===================================================================
--- branches/7.7.x/engine/src/main/resources/org/teiid/query/i18n.properties	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/main/resources/org/teiid/query/i18n.properties	2013-11-04 19:45:55 UTC (rev 4609)
@@ -191,7 +191,7 @@
 ValidationVisitor.limit_not_valid_for_xml=The limit clause cannot be used on an XML document query.
 ValidationVisitor.union_insert = Select into is not allowed under a set operation: {0}.
 ValidationVisitor.multisource_insert = A multi-source table, {0}, cannot be used in an INSERT with query expression or SELECT INTO statement.
-ValidationVisitor.invalid_encoding = Invalid encoding: {0}.
+ValidationVisitor.invalid_encoding = Encoding {0} is not valid.
 ValidationVisitor.nonUpdatable = The specified change set {0} against an inherently updatable view does not map to a key preserving group.
 ValidationVisitor.insert_qe_partition = Inserts with query expressions cannot be performed against a partitioned UNION view {0}.
 ValidationVisitor.insert_no_partition = Could not determine INSERT target for a partitioned UNION view {0} with values {1}.
@@ -279,10 +279,9 @@
 SystemSource.array_get_desc=Get the object value at the given array index
 SystemSource.array_get_param2=Array index
 SystemSource.array_get_result=The object value
-SystemSource.Add_desc=Converts escape sequences in the given string to their actual characters. 
+SystemSource.Add_desc=Add two numbers
 SystemSource.unescape_param1=String to be unescaped
 SystemSource.unescape_result=Unescaped string
-SystemSource.Add_desc=Add two numbers
 SystemSource.Add_result_desc=Left operand + right operand
 SystemSource.Subtract_desc=Subtract two numbers
 SystemSource.Subtract_result_desc=Left operand - right operand
@@ -608,9 +607,8 @@
 SystemSource.xsltransform_param2=XSL stylesheet
 SystemSource.xsltransform_result=Clob result
 SystemSource.xmlconcat_description=Concat XML types. 
-SystemSource.xmlconcat_param1=First element
+SystemSource.xmlconcat_param1=String comment
 SystemSource.xmlcomment_description=Create an XML comment. 
-SystemSource.xmlconcat_param1=String comment
 SystemSource.xmlconcat_result=XML result
 SystemSource.xmlpi_description=Create an XML processing instruction. 
 SystemSource.xmlpi_param1=Target
@@ -732,7 +730,6 @@
 ValidationVisitor.invalid_default=XMLTABLE DEFAULT expression is invalid: "{0}"
 ValidationVisitor.context_required=The XQuery requires a context item, but none exists in the PASSING clause.
 ValidationVisitor.xmlparse_type=XMLPARSE expects a STRING, CLOB, or BLOB value.
-ValidationVisitor.invalid_encoding=Encoding {0} is not valid.
 ValidationVisitor.subquery_insert=SELECT INTO should not be used in a subquery.
 UpdateProcedureResolver.only_variables=Variable "{0}" is read only and cannot be assigned a value.
 MappingLoader.unknown_node_type=Unknown Node Type "{0}" being loaded by the XML mapping document.
@@ -786,6 +783,7 @@
 ExecDynamicSqlInstruction.3=There is a recursive invocation of group ''{0}''. Please correct the SQL.
 ExecDynamicSqlInstruction.4=The dynamic sql string contains an incorrect number of elements.
 ExecDynamicSqlInstruction.6=The datatype ''{0}'' for element ''{1}'' in the dynamic SQL cannot be implicitly converted to ''{2}''.
+
 ExecDynamicSqlInstruction.couldnt_execute=Couldn''t execute the dynamic SQL command "{0}" with the SQL statement "{1}" due to: {2}
 
 RulePlanJoins.cantSatisfy=Join region with unsatisfied access patterns cannot be satisfied by the join criteria, Access patterns: {0} 
@@ -809,14 +807,36 @@
 FileStoreageManager.error_reading=Error reading {0}
 FileStoreageManager.no_directory=No directory specified for the file storage manager.
 FileStoreageManager.not_a_directory={0} is not a valid storage manager directory.
-FileStoreageManager.space_exhausted=Max buffer space of {0} bytes has been exceed.  The current operation will be aborted.
+FileStoreageManager.space_exhausted=Max buffer space of {2} bytes has been exceed with an allocation of {0} bytes for a total of {1}.  The current operation will be aborted.
 
+TEIID30168=Couldn''t execute the dynamic SQL command "{0}" with the SQL statement "{1}" due to: {2}
+
+TEIID30229=Temporary table "{0}" already exists.
+TEIID30226=Temporary table "{0}" does not exist.
+
+TEIID30295=Cannot create a query for MappingClass with user criteria {0}
+TEIID30296=Conjunct "{0}" has no relationship with target context {1}.
+TEIID30297=Conjunct "{0}" has a non-simple relationship to its parent through context {1}.
+
+TEIID30308=Staging table criteria cannot contian context functions
+TEIID30307=Staging table criteria {0} was not specified against a single staging table
+TEIID30302=Element {0} is not in the scope of the context {1}
+TEIID30301=Element {0} is not a valid data node
+TEIID30216=Results for the mapping class {0} are not found;
+TEIID30270=No valid criteria specified for procedure parameter {0}
+TEIID30164=The procedure parameter {0} is not nullable, but is set to null.
+
+TEIID30042=Error creating {0}
+TEIID30048=Error reading {0} {1}
+TEIID30040=No directory specified for the file storage manager.
+TEIID30041={0} is not a valid storage manager directory.
+
 TextTableNode.no_value=No value found for column {0} in the row ending on text line {1} in {2}.
 TextTableNode.conversion_error=Could not convert value for column {0} in the row ending on text line {1} in {2}.
 TextTableNode.header_missing=HEADER entry missing for column name {0} in {1}. 
 TextTableNode.unclosed=Text parse error: Unclosed qualifier at end of text in {0}.
 TextTableNode.character_not_allowed=Text parse error: Non-whitespace character found between the qualifier and the delimiter in text line {0} in {1}.
-TextTableNode.unknown_escape=Text parse error: Unknown escape sequence \\{0} in text line {1} in {2}.
+TextTableNode.unknown_escape=Text parse error\: Unknown escape sequence \\{0} in text line {1} in {2}.
 TextTableNode.invalid_width=Text parse error: Fixed width line width {0} is smaller than the expected {1} on text line {2} in {3}.
 TextTableNode.line_too_long=Text parse error: Delimited line is longer than the expected max of {2} on text line {0} in {1}.
 ValidationVisitor.fixed_option=NO ROW DELIMITER can only be used in fixed parsing mode.  
@@ -884,9 +904,8 @@
 TransactionServer.existing_transaction=Client thread already involved in a transaction. Transaction nesting is not supported. The current transaction must be completed first.
 TransactionServer.no_transaction=No transaction found for client {0}.
 TransactionServer.concurrent_transaction=Concurrent enlistment in global transaction {0} is not supported.
-TransactionServer.no_global_transaction=Expected an existing global transaction {0} but there was none for client {1}
+TransactionServer.no_global_transaction=No global transaction found for {0}.
 TransactionServer.unknown_flags=Unknown flags
-TransactionServer.no_global_transaction=No global transaction found for {0}.
 TransactionServer.wrong_transaction=Client is not currently enlisted in transaction {0}.
 TransactionServer.resume_failed=Cannot resume, transaction {0} was not suspended by client {1}.
 TransactionServer.existing_global_transaction=Global transaction {0} already exists.
@@ -901,16 +920,7 @@
 TransformationMetadata.DeletePlan_could_not_be_found_for_physical_group__12=DeletePlan could not be found for physical group 
 TransformationMetadata.Error_trying_to_read_schemas_for_the_document/table____1=Error trying to read schemas for the document/table : 
 TransformationMetadata.Invalid_type=Invalid type: {0}.
-TransformationMetadata.does_not_exist._1=does not exist.
 TransformationMetadata.0={0} ambiguous, more than one entity matching the same name
-TransformationMetadata.Error_trying_to_read_virtual_document_{0},_with_body__n{1}_1=Error trying to read virtual document {0}, with body \n{1}
-TransformationMetadata.Unknown_support_constant___12=Unknown support constant: 
-TransformationMetadata.QueryPlan_could_not_be_found_for_physical_group__6=QueryPlan could not be found for physical group 
-TransformationMetadata.InsertPlan_could_not_be_found_for_physical_group__8=InsertPlan could not be found for physical group 
-TransformationMetadata.InsertPlan_could_not_be_found_for_physical_group__10=InsertPlan could not be found for physical group 
-TransformationMetadata.DeletePlan_could_not_be_found_for_physical_group__12=DeletePlan could not be found for physical group 
-TransformationMetadata.Error_trying_to_read_schemas_for_the_document/table____1=Error trying to read schemas for the document/table : 
-TransformationMetadata.Invalid_type=Invalid type: {0}.
 
 CachedFinder.no_connector_found=No connector with jndi-name {0} found for Model {1} with source name {2} 
 translator_not_found=Translator {0} not accessible.
@@ -953,4 +963,4 @@
 invalid_table=Invalid table {0}.  A table must have 1 or more columns. 
 
 query_timeout=Cancelling query {0} since it has exceeded the timeout of {1} milliseconds.
-TEIID31138=Cannot add batch to invalidated cache group "{1}".  Check prior logs to see if there was an error persisting a batch.
\ No newline at end of file
+TEIID31138=Cannot add batch to invalidated cache group "{1}".  Check prior logs to see if there was an error persisting a batch.

Modified: branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java
===================================================================
--- branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestBufferFrontedFileStoreCache.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -29,6 +29,7 @@
 import java.io.ObjectOutput;
 import java.lang.ref.WeakReference;
 
+import org.junit.After;
 import org.junit.Test;
 import org.mockito.Mockito;
 import org.teiid.common.buffer.CacheEntry;
@@ -39,6 +40,8 @@
 
 public class TestBufferFrontedFileStoreCache {
 	
+	private BufferFrontedFileStoreCache cache;
+
 	private final static class SimpleSerializer implements Serializer<Integer> {
 		@Override
 		public Integer deserialize(ObjectInput ois)
@@ -69,9 +72,15 @@
 			return false;
 		}
 	}
+	
+	@After public void teardown() {
+		if (this.cache != null) {
+			cache.shutdown();
+		}
+	}
 
 	@Test public void testAddGetMultiBlock() throws Exception {
-		BufferFrontedFileStoreCache cache = createLayeredCache(1 << 26, 1 << 26, true);
+		cache = createLayeredCache(1 << 26, 1 << 26, true);
 		
 		CacheEntry ce = new CacheEntry(2l);
 		Serializer<Integer> s = new SimpleSerializer();
@@ -150,7 +159,7 @@
 	}
 	
 	@Test public void testEviction() throws Exception {
-		BufferFrontedFileStoreCache cache = createLayeredCache(1<<15, 1<<15, true);
+		cache = createLayeredCache(1<<15, 1<<15, true);
 		assertEquals(3, cache.getMaxMemoryBlocks());
 		
 		CacheEntry ce = new CacheEntry(2l);
@@ -181,7 +190,7 @@
 	}
 	
 	@Test public void testEvictionFails() throws Exception {
-		BufferFrontedFileStoreCache cache = createLayeredCache(1<<15, 1<<15, false);
+		cache = createLayeredCache(1<<15, 1<<15, false);
 		BufferManagerImpl bmi = Mockito.mock(BufferManagerImpl.class);
 		cache.setBufferManager(bmi);
 		Serializer<Integer> s = new SimpleSerializer();
@@ -206,6 +215,7 @@
 
 	private static BufferFrontedFileStoreCache createLayeredCache(int bufferSpace, int objectSize, boolean memStorage) throws TeiidComponentException {
 		BufferFrontedFileStoreCache fsc = new BufferFrontedFileStoreCache();
+		fsc.cleanerRunning.set(true); //prevent asynch affects
 		fsc.setMemoryBufferSpace(bufferSpace);
 		fsc.setMaxStorageObjectSize(objectSize);
 		fsc.setDirect(false);
@@ -227,7 +237,7 @@
 						
 						@Override
 						public void setLength(long length) throws IOException {
-							throw new IOException();
+							throw new OutOfDiskException(null);
 						}
 						
 						@Override
@@ -276,4 +286,101 @@
 		info.setSize(1 + (1<<13));
 	}
 	
+	@Test public void testDefragTruncateEmpty() throws Exception {
+		cache = createLayeredCache(1<<15, 1<<15, true);
+		cache.setMinDefrag(10000000);
+		Serializer<Integer> s = new SimpleSerializer();
+		WeakReference<? extends Serializer<?>> ref = new WeakReference<Serializer<?>>(s);
+		cache.createCacheGroup(s.getId());
+		Integer cacheObject = Integer.valueOf(5000);
+
+		for (int i = 0; i < 4; i++) {
+			CacheEntry ce = new CacheEntry((long)i);
+			ce.setSerializer(ref);
+			ce.setObject(cacheObject);
+
+			cache.addToCacheGroup(s.getId(), ce.getId());
+			cache.add(ce, s);
+		}
+		assertEquals(98304, cache.getDiskUsage());
+		for (int i = 0; i < 4; i++) {
+			cache.remove(1l, (long)i);
+		}
+		assertEquals(98304, cache.getDiskUsage());
+		cache.setMinDefrag(0);
+		cache.defragTask.run();
+		assertEquals(98304, cache.getDiskUsage());
+		cache.setTruncateInterval(1);
+		cache.defragTask.run();
+		assertEquals(0, cache.getDiskUsage());
+	}
+	
+	@Test public void testDefragTruncate() throws Exception {
+		cache = createLayeredCache(1<<15, 1<<15, true);
+		cache.setMinDefrag(10000000);
+		Serializer<Integer> s = new SimpleSerializer();
+		WeakReference<? extends Serializer<?>> ref = new WeakReference<Serializer<?>>(s);
+		cache.createCacheGroup(s.getId());
+		Integer cacheObject = Integer.valueOf(5000);
+
+		for (int i = 0; i < 30; i++) {
+			CacheEntry ce = new CacheEntry((long)i);
+			ce.setSerializer(ref);
+			ce.setObject(cacheObject);
+
+			cache.addToCacheGroup(s.getId(), ce.getId());
+			cache.add(ce, s);
+		}
+		assertEquals(950272, cache.getDiskUsage());
+		for (int i = 0; i < 25; i++) {
+			cache.remove(1l, (long)i);
+		}
+		assertEquals(950272, cache.getDiskUsage());
+		cache.setMinDefrag(0);
+		cache.setTruncateInterval(1);
+		cache.defragTask.run();
+		assertEquals(622592, cache.getDiskUsage());
+		cache.defragTask.run();
+		assertEquals(262144, cache.getDiskUsage());
+		cache.defragTask.run();
+		assertEquals(131072, cache.getDiskUsage());
+		cache.defragTask.run();
+		//we've reached a stable size
+		assertEquals(131072, cache.getDiskUsage());
+	}
+	
+	@Test public void testDefragMin() throws Exception {
+		cache = createLayeredCache(1<<15, 1<<15, true);
+		cache.setMinDefrag(10000000);
+		Serializer<Integer> s = new SimpleSerializer();
+		WeakReference<? extends Serializer<?>> ref = new WeakReference<Serializer<?>>(s);
+		cache.createCacheGroup(s.getId());
+		Integer cacheObject = Integer.valueOf(5000);
+
+		for (int i = 0; i < 100; i++) {
+			CacheEntry ce = new CacheEntry((long)i);
+			ce.setSerializer(ref);
+			ce.setObject(cacheObject);
+
+			cache.addToCacheGroup(s.getId(), ce.getId());
+			cache.add(ce, s);
+		}
+		assertEquals(3244032, cache.getDiskUsage());
+		for (int i = 0; i < 90; i++) {
+			cache.remove(1l, (long)i);
+		}
+		assertEquals(3244032, cache.getDiskUsage());
+		cache.setMinDefrag(5000);
+		cache.setTruncateInterval(1);
+		cache.defragTask.run();
+		assertEquals(1802240, cache.getDiskUsage());
+		cache.defragTask.run();
+		assertEquals(1114112, cache.getDiskUsage());
+		cache.defragTask.run();
+		assertEquals(655360, cache.getDiskUsage());
+		cache.defragTask.run(); 
+		//we've reached a stable size
+		assertEquals(655360, cache.getDiskUsage());
+	}
+	
 }

Modified: branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java
===================================================================
--- branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestConcurrentBitSet.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -111,4 +111,12 @@
 		
 	}
 	
+	@Test public void testCompactHighestEmpty() {
+		ConcurrentBitSet bst = new ConcurrentBitSet(1 << 19, 1);
+		bst.setCompact(true);
+		bst.getAndSetNextClearBit();
+		bst.clear(0);
+		assertEquals(-1, bst.compactHighestBitSet(0));
+	}
+	
 }

Modified: branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestFileStorageManager.java
===================================================================
--- branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestFileStorageManager.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/engine/src/test/java/org/teiid/common/buffer/impl/TestFileStorageManager.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -40,7 +40,7 @@
 @SuppressWarnings("nls")
 public class TestFileStorageManager {
 		
-	public FileStorageManager getStorageManager(Integer openFiles, String dir) throws TeiidComponentException {
+	public static FileStorageManager getStorageManager(Integer openFiles, String dir) throws TeiidComponentException {
         FileStorageManager sm = new FileStorageManager();
         sm.setStorageDirectory(UnitTestUtil.getTestScratchPath() + (dir != null ? File.separator + dir : "")); //$NON-NLS-1$
         if (openFiles != null) {

Modified: branches/7.7.x/test-integration/perf/src/test/java/org/teiid/query/eval/TestEnginePerformance.java
===================================================================
--- branches/7.7.x/test-integration/perf/src/test/java/org/teiid/query/eval/TestEnginePerformance.java	2013-10-30 19:34:17 UTC (rev 4608)
+++ branches/7.7.x/test-integration/perf/src/test/java/org/teiid/query/eval/TestEnginePerformance.java	2013-11-04 19:45:55 UTC (rev 4609)
@@ -44,7 +44,9 @@
 import javax.xml.stream.XMLStreamWriter;
 
 import org.junit.BeforeClass;
+import org.junit.FixMethodOrder;
 import org.junit.Test;
+import org.junit.runners.MethodSorters;
 import org.teiid.api.exception.query.QueryParserException;
 import org.teiid.client.BatchSerializer;
 import org.teiid.common.buffer.BlockedException;
@@ -86,6 +88,7 @@
 import org.teiid.query.unittest.RealMetadataFactory;
 import org.teiid.query.util.CommandContext;
 
+ at FixMethodOrder(MethodSorters.JVM)
 @SuppressWarnings("nls")
 public class TestEnginePerformance {
 	



More information about the teiid-commits mailing list