Author: shawkins
Date: 2011-06-07 17:36:58 -0400 (Tue, 07 Jun 2011)
New Revision: 3231
Added:
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java
branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java
Removed:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/NodeTestUtil.java
Modified:
branches/7.4.x/build/kits/jboss-container/deploy/teiid/teiid-jboss-beans.xml
branches/7.4.x/build/kits/jboss-container/teiid-releasenotes.html
branches/7.4.x/console/src/main/resources/META-INF/rhq-plugin.xml
branches/7.4.x/documentation/admin-guide/src/main/docbook/en-US/content/performance.xml
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/BufferManager.java
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/TupleBatch.java
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/TupleBuffer.java
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
branches/7.4.x/engine/src/main/java/org/teiid/query/processor/relational/GroupingNode.java
branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/BufferManagerFactory.java
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestAggregateProcessing.java
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestProcessor.java
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestGroupingNode.java
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestJoinNode.java
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestProjectIntoNode.java
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestSortNode.java
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestUnionAllNode.java
branches/7.4.x/runtime/src/main/java/org/teiid/deployers/PgCatalogMetadataStore.java
branches/7.4.x/runtime/src/main/java/org/teiid/services/BufferServiceImpl.java
branches/7.4.x/runtime/src/test/java/org/teiid/dqp/service/buffer/TestLocalBufferService.java
Log:
TEIID-1624 exposed the reserved/processing memory settings as kb. updated the admin
guide. also fixed a type issue with aggregate order by processing
Modified: branches/7.4.x/build/kits/jboss-container/deploy/teiid/teiid-jboss-beans.xml
===================================================================
---
branches/7.4.x/build/kits/jboss-container/deploy/teiid/teiid-jboss-beans.xml 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/build/kits/jboss-container/deploy/teiid/teiid-jboss-beans.xml 2011-06-07
21:36:58 UTC (rev 3231)
@@ -25,15 +25,15 @@
<!-- The max row count of a batch from a connector. Should be even multiple of
processorBatchSize. (default 1024) -->
<property name="connectorBatchSize">1024</property>
<!--
- The number of batch columns to allow in buffer memory. -1 means to
automatically calculate a value (default -1).
+ The approximate amount of buffer memory in kilobytes allowable for a single
processing operation (sort, grouping, etc.) regardless of existing memory commitments. -1
means to automatically calculate a value (default -1).
See the admin guide for more.
-->
- <property name="maxReserveBatchColumns">-1</property>
+ <property name="maxReserveKb">-1</property>
<!--
- The number of batch columns guaranteed to a processing operation. -1 means to
automatically calculate a value (default -1).
+ The approximate amount of memory in kilobytes allowed to be held by the
buffer manager. -1 means to automatically calculate a value (default -1).
See the admin guide for more.
-->
- <property name="maxProcessingBatchesColumns">-1</property>
+ <property name="maxProcessingKb">-1</property>
<!-- Max File size in MB (default 2GB)-->
<property name="maxFileSize">2048</property>
<!-- Max storage space, in MB, to be used for buffer files (default 50G)
-->
Modified: branches/7.4.x/build/kits/jboss-container/teiid-releasenotes.html
===================================================================
--- branches/7.4.x/build/kits/jboss-container/teiid-releasenotes.html 2011-06-07 21:36:50
UTC (rev 3230)
+++ branches/7.4.x/build/kits/jboss-container/teiid-releasenotes.html 2011-06-07 21:36:58
UTC (rev 3231)
@@ -130,6 +130,12 @@
See the <a href="teiid-docs/teiid_admin_guide.pdf">Admin Guide</a>
for more on configuration and installation.
+<h4>from 7.4</h4>
+<ul>
+ <LI>The configuration for authorization has been moved off of the
RuntimeEngineDeployer bean and onto separate AuthorizationValidator and PolicyDecider
beans.
+ <LI>The configuration for the buffer manager has been simplified to refer to
memory sizes in KB, rather than batch columns.
+</ul>
+
<h4>from 7.3</h4>
<ul>
<LI>The default value for the JDBC dynamic vdb importer setting
importer.useFullSchemaName is now true, which matches the expected behavior from the
documentation.
Modified: branches/7.4.x/console/src/main/resources/META-INF/rhq-plugin.xml
===================================================================
--- branches/7.4.x/console/src/main/resources/META-INF/rhq-plugin.xml 2011-06-07 21:36:50
UTC (rev 3230)
+++ branches/7.4.x/console/src/main/resources/META-INF/rhq-plugin.xml 2011-06-07 21:36:58
UTC (rev 3231)
@@ -387,16 +387,16 @@
displayName="Connector Batch Size"
description="The max row count of a batch from a connector. Should be even
multiple of processorBatchSize. (default 1024)"
required="false" readOnly="false" />
- <c:simple-property name="BufferService.maxProcessingBatchesColumns"
- displayName="Max Processing Batches Columns"
- description="The number of batch columns guarenteed to a processing operation.
Set this value lower if the workload typically processes larger numbers of concurrent
queries with large intermediate results from operations such as sorting, grouping, etc.
(default 128)"
+ <c:simple-property name="BufferService.maxProcessingKb"
+ displayName="Max Processing Memory"
+ description="The approximate amount of buffer memory in kilobytes allowable for
a single processing operation (sort, grouping, etc.) regardless of existing memory
commitments. -1 means to automatically calculate a value (default -1)."
required="false" readOnly="false" />
<c:simple-property name="BufferService.maxFileSize"
displayName="Max File Size"
description="Max file size for buffer files (default 2GB)"
required="false" readOnly="false" />
- <c:simple-property name="BufferService.maxReserveBatchColumns"
- displayName="Max Reserve Batch Columns"
- description="The number of batch columns to allow in memory (default 16384).
This value should be set lower or higher depending on the available memory to Teiid in the
VM. 16384 is considered a good default for a dedicated 32-bit VM running Teiid with a 1
gig heap."
+ <c:simple-property name="BufferService.maxReserveKb"
+ displayName="Max Reserve Memory"
+ description="The approximate amount of memory in kilobytes allowed to be held
by the buffer manager. -1 means to automatically calculate a value (default -1)."
required="false" readOnly="false" />
</c:group>
<c:group name="JdbcSocketConfiguration" displayName="Jdbc Socket
Configuration Properties" hiddenByDefault="false">
Modified:
branches/7.4.x/documentation/admin-guide/src/main/docbook/en-US/content/performance.xml
===================================================================
---
branches/7.4.x/documentation/admin-guide/src/main/docbook/en-US/content/performance.xml 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/documentation/admin-guide/src/main/docbook/en-US/content/performance.xml 2011-06-07
21:36:58 UTC (rev 3231)
@@ -18,12 +18,10 @@
accessing the Teiid server simultaneously.
</para>
<para>
- The <code>maxReserveBatchColumns</code>
- setting determines the total size of batches that can be held by the BufferManager in
memory.
- This number does not include persistent batches held by soft (such as
+ The <code>maxReserveKb</code>
+ setting determines the total size in kilobytes of batches that can be held by the
BufferManager in memory.
+ This number does not account for persistent batches held by soft (such as
index pages) or weak references.
- The value is treated internally as an approximation of bytes using the conversion
- <code>maxReserveBatchColumns</code> *
<code>processorBatchSize</code> * (64bytes per column value).
The default value of -1 will auto-calculate a typical max based upon the max heap
available to the VM.
The auto-calculated value assumes a 64bit architecture and will limit buffer usage to
50% of the first
gigabyte of memory beyond the first 300 megabytes (which are assumed for use by the AS
and other Teiid purposes)
@@ -39,22 +37,21 @@
</para>
<note>
<para>Memory consumption can be significantly more or less than the nominal
target
- depending upon actual column values and whether value caching is enabled.
Large strings, bigintegers, bigdecimals, or values typed as object can exceed their
default size estimate.
- If an out of memory errors occur, then set a lower the
maxReserveBatchColumns value.
+ depending upon actual column values and whether value caching is enabled.
Large non built-in type objects can exceed their default size estimate.
+ If an out of memory errors occur, then set a lower the maxReserveKb value.
Also note that source lob values are held by memory references that are not cleared when a
batch is persisted.
+ With heavy lob usage you should ensure that buffers of other memory
associated with lob references are appropiately sized.
</para>
</note>
<para>
- The <code>maxProcessingBatchesColumns</code>
- setting determines the total size of batches that can be used by active plans
regardless of the memory held based on <code>maxReserveBatchColumns</code>.
- The value is treated internally as an approximation of bytes using the conversion
- <code>maxProcessingBatchesColumns</code> *
<code>processorBatchSize</code> * (64bytes per column value).
+ The <code>maxProcessingKb</code>
+ setting determines the total size in kilobytes of batches that can be used by active
plans regardless of the memory held based on <code>maxReserveKb</code>.
The default value of -1 will auto-calculate a typical max based upon the max heap
available to the VM and max active plans.
The auto-calculated value assumes a 64bit architecture and will limit processing batch
usage to 10% of memory
beyond the first 300 megabytes (which are assumed for use by the AS and other Teiid
purposes).
</para>
<para>
- In systems where large intermediate results are normal (scrolling cursors or sorting
over millions of rows) you can consider increasing the
<code>maxProcessingBatchColumns</code> and decreasing
- the <code>maxReserveBatchColumns</code> so that each request has access to
an effectively smaller buffer space.
+ In systems where large intermediate results are normal (scrolling cursors or sorting
over millions of rows) you can consider increasing the
<code>maxProcessingKb</code> and decreasing
+ the <code>maxReserveKb</code> so that each request has access to an
effectively smaller buffer space.
</para>
<para>
Each intermediate result buffer, temporary LOB, and temporary table
Modified: branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/BufferManager.java
===================================================================
---
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/BufferManager.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/BufferManager.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -67,8 +67,8 @@
public static int DEFAULT_CONNECTOR_BATCH_SIZE = 1024;
public static int DEFAULT_PROCESSOR_BATCH_SIZE = 512;
- public static int DEFAULT_MAX_PROCESSING_BATCHES = -1;
- public static int DEFAULT_RESERVE_BUFFERS = -1;
+ public static int DEFAULT_MAX_PROCESSING_KB = -1;
+ public static int DEFAULT_RESERVE_BUFFER_KB = -1;
/**
* Get the batch size to use during query processing.
Modified: branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/TupleBatch.java
===================================================================
--- branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/TupleBatch.java 2011-06-07
21:36:50 UTC (rev 3230)
+++ branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/TupleBatch.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -156,6 +156,10 @@
this.types = types;
}
+ public String[] getDataTypes() {
+ return types;
+ }
+
public boolean containsRow(int row) {
return rowOffset <= row && getEndRow() >= row;
}
Modified: branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/TupleBuffer.java
===================================================================
---
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/TupleBuffer.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/TupleBuffer.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -365,4 +365,8 @@
return prefersMemory;
}
+ public String[] getTypes() {
+ return types;
+ }
+
}
Modified:
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java
===================================================================
---
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/BufferManagerImpl.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -33,6 +33,7 @@
import java.lang.ref.SoftReference;
import java.lang.ref.WeakReference;
import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.Iterator;
@@ -64,6 +65,7 @@
import org.teiid.dqp.internal.process.DQPConfiguration;
import org.teiid.logging.LogConstants;
import org.teiid.logging.LogManager;
+import org.teiid.logging.MessageLevel;
import org.teiid.query.QueryPlugin;
import org.teiid.query.processor.relational.ListNestedSortComparator;
import org.teiid.query.sql.symbol.Expression;
@@ -87,25 +89,24 @@
*/
public class BufferManagerImpl implements BufferManager, StorageManager {
- public static final double KB_PER_VALUE = 64d/1024;
private static final int IO_BUFFER_SIZE = 1 << 14;
private static final int COMPACTION_THRESHOLD = 1 << 25; //start checking at 32
megs
private final class BatchManagerImpl implements BatchManager {
private final String id;
- private final int columnCount;
private volatile FileStore store;
private Map<Long, long[]> physicalMapping = new ConcurrentHashMap<Long,
long[]>();
private ReadWriteLock compactionLock = new ReentrantReadWriteLock();
private AtomicLong unusedSpace = new AtomicLong();
private int[] lobIndexes;
+ private SizeUtility sizeUtility;
- private BatchManagerImpl(String newID, int columnCount, int[] lobIndexes) {
+ private BatchManagerImpl(String newID, int[] lobIndexes) {
this.id = newID;
- this.columnCount = columnCount;
this.store = createFileStore(id);
this.store.setCleanupReference(this);
this.lobIndexes = lobIndexes;
+ this.sizeUtility = new SizeUtility();
}
public FileStore createStorage(String prefix) {
@@ -185,7 +186,7 @@
ManagedBatchImpl removeBatch(int row) {
ManagedBatchImpl result = batches.remove(row);
if (result != null) {
- activeBatchColumnCount -= result.batchManager.columnCount;
+ activeBatchKB -= result.sizeEstimate;
}
return result;
}
@@ -200,17 +201,19 @@
private BatchManagerImpl batchManager;
private long id;
private LobManager lobManager;
+ private int sizeEstimate;
public ManagedBatchImpl(TupleBatch batch, BatchManagerImpl manager, boolean softCache)
{
this.softCache = softCache;
id = batchAdded.incrementAndGet();
- LogManager.logTrace(LogConstants.CTX_BUFFER_MGR, "Add batch to
BufferManager", id); //$NON-NLS-1$
this.activeBatch = batch;
this.beginRow = batch.getBeginRow();
this.batchManager = manager;
if (this.batchManager.lobIndexes != null) {
this.lobManager = new LobManager();
}
+ sizeEstimate = (int) Math.max(1, manager.sizeUtility.getBatchSize(batch) / 1024);
+ LogManager.logTrace(LogConstants.CTX_BUFFER_MGR, "Add batch to
BufferManager", id, "with size estimate", sizeEstimate); //$NON-NLS-1$
//$NON-NLS-2$
}
@Override
@@ -225,7 +228,7 @@
if (batch == null) {
return; //already removed
}
- activeBatchColumnCount += batchManager.columnCount;
+ activeBatchKB += sizeEstimate;
TupleBufferInfo tbi = null;
if (update) {
tbi = activeBatches.remove(batchManager.id);
@@ -287,6 +290,7 @@
try {
this.batchManager.compactionLock.readLock().lock();
long[] info = batchManager.physicalMapping.get(this.id);
+ Assertion.isNotNull(info, "Invalid batch " + id); //$NON-NLS-1$
ObjectInputStream ois = new ObjectInputStream(new
BufferedInputStream(batchManager.store.createInputStream(info[0]), IO_BUFFER_SIZE));
batch = new TupleBatch();
batch.setDataTypes(types);
@@ -341,7 +345,7 @@
}
if (softCache) {
this.batchReference = new SoftReference<TupleBatch>(batch);
- } else {
+ } else if (useWeakReferences) {
this.batchReference = new WeakReference<TupleBatch>(batch);
}
}
@@ -385,22 +389,21 @@
private int connectorBatchSize = BufferManager.DEFAULT_CONNECTOR_BATCH_SIZE;
private int processorBatchSize = BufferManager.DEFAULT_PROCESSOR_BATCH_SIZE;
//set to acceptable defaults for testing
- private int maxProcessingBatches = 128;
- private int maxReserveBatchColumns = 16384;
- private int maxProcessingKB;
- private int maxReserveBatchKB;
+ private int maxProcessingKB = 1 << 11;
+ private Integer maxProcessingKBOrig;
+ private int maxReserveKB = 1 << 25;
private volatile int reserveBatchKB;
private int maxActivePlans = DQPConfiguration.DEFAULT_MAX_ACTIVE_PLANS; //used as a
hint to set the reserveBatchKB
+ private boolean useWeakReferences = true;
private ReentrantLock lock = new ReentrantLock(true);
private Condition batchesFreed = lock.newCondition();
- private volatile int activeBatchColumnCount = 0;
+ private volatile int activeBatchKB = 0;
private Map<String, TupleBufferInfo> activeBatches = new
LinkedHashMap<String, TupleBufferInfo>();
private Map<String, TupleReference> tupleBufferMap = new
ConcurrentHashMap<String, TupleReference>();
private ReferenceQueue<TupleBuffer> tupleBufferQueue = new
ReferenceQueue<TupleBuffer>();
-
private StorageManager diskMgr;
private AtomicLong tsId = new AtomicLong();
@@ -431,10 +434,6 @@
return maxProcessingKB;
}
- public void setMaxProcessingBatchColumns(int maxProcessingBatches) {
- this.maxProcessingBatches = maxProcessingBatches;
- }
-
/**
* Get processor batch size
* @return Number of rows in a processor batch
@@ -480,17 +479,19 @@
TupleSourceType tupleSourceType) {
final String newID = String.valueOf(this.tsId.getAndIncrement());
int[] lobIndexes = LobManager.getLobIndexes(elements);
- BatchManager batchManager = new BatchManagerImpl(newID, elements.size(),
lobIndexes);
+ BatchManager batchManager = new BatchManagerImpl(newID, lobIndexes);
TupleBuffer tupleBuffer = new TupleBuffer(batchManager, newID, elements,
lobIndexes, getProcessorBatchSize());
- LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Creating
TupleBuffer:", newID, "of type ", tupleSourceType); //$NON-NLS-1$
//$NON-NLS-2$
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR,
MessageLevel.DETAIL)) {
+ LogManager.logDetail(LogConstants.CTX_BUFFER_MGR, "Creating
TupleBuffer:", newID, elements, Arrays.toString(tupleBuffer.getTypes()), "of
type", tupleSourceType); //$NON-NLS-1$ //$NON-NLS-2$
+ }
return tupleBuffer;
}
public STree createSTree(final List elements, String groupName, int keyLength) {
String newID = String.valueOf(this.tsId.getAndIncrement());
int[] lobIndexes = LobManager.getLobIndexes(elements);
- BatchManager bm = new BatchManagerImpl(newID, elements.size(), lobIndexes);
- BatchManager keyManager = new
BatchManagerImpl(String.valueOf(this.tsId.getAndIncrement()), keyLength, null);
+ BatchManager bm = new BatchManagerImpl(newID, lobIndexes);
+ BatchManager keyManager = new
BatchManagerImpl(String.valueOf(this.tsId.getAndIncrement()), null);
int[] compareIndexes = new int[keyLength];
for (int i = 1; i < compareIndexes.length; i++) {
compareIndexes[i] = i;
@@ -509,27 +510,35 @@
this.maxActivePlans = maxActivePlans;
}
+ public void setMaxProcessingKB(int maxProcessingKB) {
+ this.maxProcessingKB = maxProcessingKB;
+ }
+
+ public void setMaxReserveKB(int maxReserveBatchKB) {
+ this.maxReserveKB = maxReserveBatchKB;
+ }
+
@Override
public void initialize() throws TeiidComponentException {
int maxMemory = (int)Math.min(Runtime.getRuntime().maxMemory() / 1024,
Integer.MAX_VALUE);
maxMemory -= 300 * 1024; //assume 300 megs of overhead for the AS/system stuff
- if (maxReserveBatchColumns < 0) {
- this.maxReserveBatchKB = 0;
+ if (maxReserveKB < 0) {
+ this.maxReserveKB = 0;
int one_gig = 1024 * 1024;
if (maxMemory > one_gig) {
//assume 75% of the memory over the first gig
- this.maxReserveBatchKB += (int)Math.max(0, (maxMemory - one_gig) * .75);
+ this.maxReserveKB += (int)Math.max(0, (maxMemory - one_gig) * .75);
}
- this.maxReserveBatchKB += Math.max(0, Math.min(one_gig, maxMemory) * .5);
- } else {
- this.maxReserveBatchKB = Math.max(0, (int)Math.min(maxReserveBatchColumns *
KB_PER_VALUE * processorBatchSize, Integer.MAX_VALUE));
+ this.maxReserveKB += Math.max(0, Math.min(one_gig, maxMemory) * .5);
}
- this.reserveBatchKB = this.maxReserveBatchKB;
- if (this.maxProcessingBatches < 0) {
- this.maxProcessingKB = Math.max((int)Math.min(128 * KB_PER_VALUE * processorBatchSize,
Integer.MAX_VALUE), (int)(.1 * maxMemory)/maxActivePlans);
- } else {
- this.maxProcessingKB = Math.max(0, (int)Math.min(Math.ceil(maxProcessingBatches *
KB_PER_VALUE * processorBatchSize), Integer.MAX_VALUE));
+ this.reserveBatchKB = this.maxReserveKB;
+ if (this.maxProcessingKBOrig == null) {
+ //store the config value so that we can be reinitialized (this is not a clean
approach)
+ this.maxProcessingKBOrig = this.maxProcessingKB;
}
+ if (this.maxProcessingKBOrig < 0) {
+ this.maxProcessingKB = Math.max(Math.min(8 * processorBatchSize, Integer.MAX_VALUE),
(int)(.1 * maxMemory)/maxActivePlans);
+ }
}
@Override
@@ -537,6 +546,9 @@
if (count < 1) {
return;
}
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR,
MessageLevel.TRACE)) {
+ LogManager.logTrace(LogConstants.CTX_BUFFER_MGR, "Releasing buffer
space", count); //$NON-NLS-1$
+ }
lock.lock();
try {
this.reserveBatchKB += count;
@@ -548,11 +560,14 @@
@Override
public int reserveBuffers(int count, BufferReserveMode mode) {
+ if (LogManager.isMessageToBeRecorded(LogConstants.CTX_BUFFER_MGR,
MessageLevel.TRACE)) {
+ LogManager.logTrace(LogConstants.CTX_BUFFER_MGR, "Reserving buffer
space", count, mode); //$NON-NLS-1$
+ }
lock.lock();
try {
if (mode == BufferReserveMode.WAIT) {
//don't wait for more than is available
- int waitCount = Math.min(count, this.maxReserveBatchKB);
+ int waitCount = Math.min(count, this.maxReserveKB);
while (waitCount > 0 && waitCount > this.reserveBatchKB) {
try {
batchesFreed.await(100, TimeUnit.MILLISECONDS);
@@ -576,13 +591,13 @@
}
void persistBatchReferences() {
- if (activeBatchColumnCount == 0 || activeBatchColumnCount <= reserveBatchKB) {
- int memoryCount = activeBatchColumnCount + maxReserveBatchColumns - reserveBatchKB;
+ if (activeBatchKB == 0 || activeBatchKB <= reserveBatchKB) {
+ int memoryCount = activeBatchKB + maxReserveKB - reserveBatchKB;
if (DataTypeManager.isValueCacheEnabled()) {
- if (memoryCount < maxReserveBatchColumns / 8) {
+ if (memoryCount < maxReserveKB / 8) {
DataTypeManager.setValueCacheEnabled(false);
}
- } else if (memoryCount > maxReserveBatchColumns / 4) {
+ } else if (memoryCount > maxReserveKB / 4) {
DataTypeManager.setValueCacheEnabled(true);
}
return;
@@ -590,7 +605,7 @@
while (true) {
ManagedBatchImpl mb = null;
synchronized (activeBatches) {
- if (activeBatchColumnCount == 0 || activeBatchColumnCount * 5 < reserveBatchKB *
4) {
+ if (activeBatchKB == 0 || activeBatchKB < reserveBatchKB * .8) {
break;
}
Iterator<TupleBufferInfo> iter = activeBatches.values().iterator();
@@ -624,43 +639,12 @@
//this includes alignment, row/array, and reference overhead
for (Expression element : elements) {
Class<?> type = element.getType();
- if (type == DataTypeManager.DefaultDataClasses.STRING) {
- total += isValueCacheEnabled?100:256; //assumes an "average" string length
of approximately 100 chars
- } else if (type == DataTypeManager.DefaultDataClasses.DATE
- || type == DataTypeManager.DefaultDataClasses.TIME
- || type == DataTypeManager.DefaultDataClasses.TIMESTAMP) {
- total += isValueCacheEnabled?20:28;
- } else if (type == DataTypeManager.DefaultDataClasses.LONG
- || type == DataTypeManager.DefaultDataClasses.DOUBLE) {
- total += isValueCacheEnabled?12:16;
- } else if (type == DataTypeManager.DefaultDataClasses.INTEGER
- || type == DataTypeManager.DefaultDataClasses.FLOAT) {
- total += isValueCacheEnabled?6:12;
- } else if (type == DataTypeManager.DefaultDataClasses.CHAR
- || type == DataTypeManager.DefaultDataClasses.SHORT) {
- total += isValueCacheEnabled?4:10;
- } else if (type == DataTypeManager.DefaultDataClasses.OBJECT) {
- total += 1024;
- } else if (type == DataTypeManager.DefaultDataClasses.NULL) {
- //it's free
- } else if (type == DataTypeManager.DefaultDataClasses.BYTE) {
- total += 2; //always value cached
- } else if (type == DataTypeManager.DefaultDataClasses.BOOLEAN) {
- total += 1; //always value cached
- } else {
- total += 512; //assumes buffer overhead in the case of lobs
- //however the account for lobs is misleading as the lob
- //references are not actually removed from memory
- }
+ total += SizeUtility.getSize(isValueCacheEnabled, type);
}
total += 8*elements.size() + 36; // column list / row overhead
total *= processorBatchSize;
return Math.max(1, total / 1024);
}
-
- public void setMaxReserveBatchColumns(int maxReserve) {
- this.maxReserveBatchColumns = maxReserve;
- }
public void shutdown() {
}
@@ -698,4 +682,9 @@
id = referent.getId();
}
}
+
+ public void setUseWeakReferences(boolean useWeakReferences) {
+ this.useWeakReferences = useWeakReferences;
+ }
+
}
Added: branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java
===================================================================
--- branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java
(rev 0)
+++
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -0,0 +1,205 @@
+/*
+ * 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.lang.reflect.Array;
+import java.math.BigDecimal;
+import java.math.BigInteger;
+
+import org.teiid.common.buffer.TupleBatch;
+import org.teiid.core.types.DataTypeManager;
+
+
+/**
+ * Utility methods to determine the size of Java objects, particularly with
+ * respect to the Teiid runtime types.
+ *
+ * The sizes are loosely based on expected heap size and are generally optimistic.
+ * Actual object allocation efficiency can be quite poor.
+ */
+public final class SizeUtility {
+ public static final int REFERENCE_SIZE = 8;
+
+ private long bigIntegerEstimate;
+ private long bigDecimalEstimate;
+
+ public SizeUtility() {
+ boolean isValueCacheEnabled = DataTypeManager.isValueCacheEnabled();
+ bigIntegerEstimate = getSize(isValueCacheEnabled,
DataTypeManager.DefaultDataClasses.BIG_INTEGER);
+ bigDecimalEstimate = getSize(isValueCacheEnabled,
DataTypeManager.DefaultDataClasses.BIG_DECIMAL);
+ }
+
+ public long getBatchSize(TupleBatch data) {
+ return getBatchSize(DataTypeManager.isValueCacheEnabled(), data);
+ }
+
+ private long getBatchSize(boolean accountForValueCache, TupleBatch data) {
+ int colLength = data.getDataTypes().length;
+ int rowLength = data.getRowCount();
+
+ // Array overhead for row array
+ long size = 16 + alignMemory(rowLength * REFERENCE_SIZE);
+ // array overhead for all the columns ( 8 object overhead + 4 ref + 4 int)
+ size += (rowLength * (48 + alignMemory(colLength * REFERENCE_SIZE)));
+ for (int col = 0; col < colLength; col++) {
+ Class<?> type =
DataTypeManager.getDataTypeClass(data.getDataTypes()[col]);
+
+ if (type == DataTypeManager.DefaultDataClasses.STRING
+ || type == DataTypeManager.DefaultDataClasses.OBJECT
+ || type == DataTypeManager.DefaultDataClasses.BIG_INTEGER
+ || type == DataTypeManager.DefaultDataClasses.BIG_DECIMAL) {
+ int estRow = 0;
+ for (int row = 0; row < rowLength; row++) {
+ boolean updateEst = row == estRow;
+ size += getSize(data.getTuples().get(row).get(col), updateEst,
accountForValueCache);
+ if (updateEst) {
+ estRow = estRow * 2 + 1;
+ }
+ }
+ } else {
+ size += getSize(accountForValueCache, type) * rowLength;
+ }
+ }
+ return size;
+ }
+
+ static int getSize(boolean isValueCacheEnabled,
+ Class<?> type) {
+ if (type == DataTypeManager.DefaultDataClasses.STRING) {
+ return isValueCacheEnabled?100:256; //assumes an "average" string length of
approximately 100 chars
+ } else if (type == DataTypeManager.DefaultDataClasses.DATE
+ || type == DataTypeManager.DefaultDataClasses.TIME
+ || type == DataTypeManager.DefaultDataClasses.TIMESTAMP) {
+ return isValueCacheEnabled?20:28;
+ } else if (type == DataTypeManager.DefaultDataClasses.LONG
+ || type == DataTypeManager.DefaultDataClasses.DOUBLE) {
+ return isValueCacheEnabled?12:16;
+ } else if (type == DataTypeManager.DefaultDataClasses.INTEGER
+ || type == DataTypeManager.DefaultDataClasses.FLOAT) {
+ return isValueCacheEnabled?6:12;
+ } else if (type == DataTypeManager.DefaultDataClasses.CHAR
+ || type == DataTypeManager.DefaultDataClasses.SHORT) {
+ return isValueCacheEnabled?4:10;
+ } else if (type == DataTypeManager.DefaultDataClasses.OBJECT) {
+ return 1024;
+ } else if (type == DataTypeManager.DefaultDataClasses.NULL) {
+ return 0; //it's free
+ } else if (type == DataTypeManager.DefaultDataClasses.BYTE
+ || type == DataTypeManager.DefaultDataClasses.BOOLEAN) {
+ return 1; //should always be value cached, but there's a small chance it's
not
+ } else if (type == DataTypeManager.DefaultDataClasses.BIG_INTEGER){
+ return isValueCacheEnabled?75:100;
+ } else if (type == DataTypeManager.DefaultDataClasses.BIG_DECIMAL) {
+ return isValueCacheEnabled?150:200;
+ }
+ return 512; //assumes buffer overhead in the case of lobs
+ //however the account for lobs is misleading as the lob
+ //references are not actually removed from memory
+ }
+
+ /**
+ * Get size of object
+ * @return Size in bytes
+ */
+ protected long getSize(Object obj, boolean updateEstimate, boolean
accountForValueCache) {
+ if(obj == null) {
+ return 0;
+ }
+
+ Class<?> type = DataTypeManager.determineDataTypeClass(obj);
+ if(type == DataTypeManager.DefaultDataClasses.STRING) {
+ int length = ((String)obj).length();
+ if (length > 0) {
+ return alignMemory(40 + (2 * length));
+ }
+ return 40;
+ } else if(obj instanceof Iterable<?>) {
+ Iterable<?> i = (Iterable<?>)obj;
+ long total = 16;
+ for (Object object : i) {
+ total += getSize(object, true, false) + REFERENCE_SIZE;
+ }
+ return total;
+ } else if(type == DataTypeManager.DefaultDataClasses.BIG_DECIMAL) {
+ if (!updateEstimate) {
+ return bigDecimalEstimate;
+ }
+ int bitLength = ((BigDecimal)obj).unscaledValue().bitLength();
+ //TODO: this does not account for the possibility of a cached string
+ long result = 88 + alignMemory(4 + (bitLength >> 3));
+ if (updateEstimate) {
+ bigDecimalEstimate = (bigDecimalEstimate + result)/2;
+ }
+ return result;
+ } else if(type == DataTypeManager.DefaultDataClasses.BIG_INTEGER) {
+ if (!updateEstimate) {
+ return bigIntegerEstimate;
+ }
+ int bitLength = ((BigInteger)obj).bitLength();
+ long result = 40 + alignMemory(4 + (bitLength >> 3));
+ if (updateEstimate) {
+ bigIntegerEstimate = (bigIntegerEstimate + result)/2;
+ }
+ return result;
+ } else if(obj.getClass().isArray()) {
+ Class<?> componentType = obj.getClass().getComponentType();
+ if (!componentType.isPrimitive()) {
+ Object[] rows = (Object[]) obj;
+ long total = 16 + alignMemory(rows.length * REFERENCE_SIZE); // Array
overhead
+ for(int i=0; i<rows.length; i++) {
+ total += getSize(rows[i], true, false);
+ }
+ return total;
+ }
+ int length = Array.getLength(obj);
+ int primitiveSize = 8;
+ if (componentType == boolean.class) {
+ primitiveSize = 4;
+ } else if (componentType == byte.class) {
+ primitiveSize = 1;
+ } else if (componentType == short.class) {
+ primitiveSize = 2;
+ } else if (componentType == int.class || componentType == float.class) {
+ primitiveSize = 4;
+ }
+ return alignMemory(length * primitiveSize) + 16;
+ }
+ return getSize(accountForValueCache, type);
+ }
+
+ /**
+ * Most current VMs have memory alignment that places objects into heap space that is
a multiple of 8 Bytes.
+ * This utility method helps with calculating the aligned size of an object.
+ * @param numBytes
+ * @return
+ * @since 4.2
+ */
+ private static long alignMemory(long numBytes) {
+ long remainder = numBytes % 8;
+ if (remainder != 0) {
+ numBytes += (8 - remainder);
+ }
+ return numBytes;
+ }
+
+}
\ No newline at end of file
Property changes on:
branches/7.4.x/engine/src/main/java/org/teiid/common/buffer/impl/SizeUtility.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Modified:
branches/7.4.x/engine/src/main/java/org/teiid/query/processor/relational/GroupingNode.java
===================================================================
---
branches/7.4.x/engine/src/main/java/org/teiid/query/processor/relational/GroupingNode.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/main/java/org/teiid/query/processor/relational/GroupingNode.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -218,7 +218,7 @@
OrderByItem item = iterator.next();
orderIndecies[iterator.previousIndex()] =
collectExpression(item.getSymbol());
element = new
ElementSymbol(String.valueOf(iterator.previousIndex()));
- element.setType(inputType);
+ element.setType(item.getSymbol().getType());
schema.add(element);
OrderByItem newItem = item.clone();
newItem.setSymbol(element);
@@ -286,7 +286,7 @@
@Override
protected List updateTuple(List tuple) throws ExpressionEvaluationException,
BlockedException, TeiidComponentException {
int columns = collectedExpressions.size();
- List exprTuple = new ArrayList(columns);
+ List<Object> exprTuple = new ArrayList<Object>(columns);
for(int col = 0; col<columns; col++) {
// The following call may throw BlockedException, but all state to this
point
// is saved in class variables so we can start over on building this
tuple
@@ -335,7 +335,7 @@
} else if(! sameGroup(currentGroupTuple, lastRow)) {
// Close old group
- List row = new ArrayList(functions.length);
+ List<Object> row = new ArrayList<Object>(functions.length);
for(int i=0; i<functions.length; i++) {
row.add( functions[i].getResult() );
functions[i].reset();
Modified:
branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/BufferManagerFactory.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/BufferManagerFactory.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/BufferManagerFactory.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -58,16 +58,35 @@
}
public static BufferManagerImpl createBufferManager() {
- BufferManagerImpl bufferMgr = new BufferManagerImpl();
- try {
- bufferMgr.initialize();
+ return initBufferManager(new BufferManagerImpl());
+ }
+
+ public static BufferManagerImpl getTestBufferManager(long bytesAvailable, int
procBatchSize, int connectorBatchSize) {
+ BufferManagerImpl bufferManager = new BufferManagerImpl();
+ bufferManager.setProcessorBatchSize(procBatchSize);
+ bufferManager.setConnectorBatchSize(connectorBatchSize);
+ bufferManager.setMaxProcessingKB((int) (bytesAvailable/1024));
+ bufferManager.setMaxReserveKB((int) (bytesAvailable/1024));
+ return initBufferManager(bufferManager);
+ }
+
+ public static BufferManagerImpl getTestBufferManager(long bytesAvailable, int
procBatchSize) {
+ BufferManagerImpl bufferManager = new BufferManagerImpl();
+ bufferManager.setProcessorBatchSize(procBatchSize);
+ bufferManager.setMaxProcessingKB((int) (bytesAvailable/1024));
+ bufferManager.setMaxReserveKB((int) (bytesAvailable/1024));
+ return initBufferManager(bufferManager);
+ }
+
+ public static BufferManagerImpl initBufferManager(BufferManagerImpl bufferManager) {
+ try {
+ bufferManager.initialize();
} catch (TeiidComponentException e) {
throw new RuntimeException(e);
}
-
- // Add unmanaged memory storage manager
- bufferMgr.setStorageManager(new MemoryStorageManager());
- return bufferMgr;
+
+ bufferManager.setStorageManager(new MemoryStorageManager());
+ return bufferManager;
}
}
Added:
branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java
===================================================================
--- branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java
(rev 0)
+++
branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -0,0 +1,185 @@
+/*
+ * 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 static org.junit.Assert.*;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.sql.Date;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.junit.Test;
+import org.teiid.common.buffer.TupleBatch;
+
+public class TestSizeUtility {
+
+ public void helpTestGetStaticSize(Object obj, long expectedSize) {
+ helpTestGetSize(obj, expectedSize);
+ }
+
+ public void helpTestGetSize(Object obj, long expectedSize) {
+ long actualSize = new SizeUtility().getSize(obj, true, false);
+ assertEquals("Got unexpected size: ", expectedSize, actualSize);
//$NON-NLS-1$
+ }
+
+ @Test public void testGetSizeChar() {
+ helpTestGetStaticSize(new Character('a'), 10);
+ }
+
+ @Test public void testGetSizeBoolean() {
+ helpTestGetStaticSize(Boolean.TRUE, 1);
+ }
+
+ @Test public void testGetSizeByte() {
+ helpTestGetStaticSize(new Byte((byte)0), 1);
+ }
+
+ @Test public void testGetSizeShort() {
+ helpTestGetStaticSize(new Short((short)0), 10);
+ }
+
+ @Test public void testGetSizeInteger() {
+ helpTestGetStaticSize(new Integer(0), 12);
+ }
+
+ @Test public void testGetSizeLong() {
+ helpTestGetStaticSize(new Long(0l), 16);
+ }
+
+ @Test public void testGetSizeFloat() {
+ helpTestGetStaticSize(new Float(0), 12);
+ }
+
+ @Test public void testGetSizeDouble() {
+ helpTestGetStaticSize(new Double(0), 16);
+ }
+
+ @Test public void testGetSizeTimestamp() {
+ helpTestGetStaticSize(new Timestamp(12301803), 28);
+ }
+
+ @Test public void testGetSizeDate() {
+ helpTestGetStaticSize(new Date(12301803), 28);
+ }
+
+ @Test public void testGetSizeTime() {
+ helpTestGetStaticSize(new Time(12301803), 28);
+ }
+
+ @Test public void testGetSizeEmptyString() {
+ helpTestGetSize("", 40); //$NON-NLS-1$
+ }
+
+ @Test public void testGetSizeShortString() {
+ helpTestGetSize("abcdefghij", 64); //$NON-NLS-1$
+ }
+
+ public void XtestGetSizeLongString() {
+ // There is no clear way of figuring out the actual size of a string that is
created
+ // from a StringBuffer because the buffer can sometimes be twice as big as the
actual length of the string
+ // Since the data comin from the connector is not created this way, this test is
an inaccurate setup
+ int size = 10000;
+ StringBuffer str = new StringBuffer();
+ for(int i=0; i<size; i++) {
+ str.append("a"); //$NON-NLS-1$
+ }
+ helpTestGetSize(str.toString(), size+3);
+ }
+
+ @Test public void testGetSizeRow1() {
+ List<Object> row = new ArrayList<Object>(1);
+ row.add(new Integer(0));
+ helpTestGetStaticSize(row, 36);
+ }
+
+ @Test public void testGetSizeRow2() {
+ List<Object> row = new ArrayList<Object>(4);
+ row.add(new Integer(0));
+ row.add(new Integer(101));
+ row.add(Boolean.TRUE);
+ row.add(new Double(1091203.00));
+ helpTestGetStaticSize(row, 89);
+ }
+
+ @Test public void testGetSizeRows1() {
+ helpTestGetStaticSize(new List[] { }, 16);
+ }
+
+ @Test public void testGetSizeRows2() {
+ List<Object> row1 = new ArrayList<Object>(2);
+ row1.add(new Integer(0));
+ row1.add(new Integer(100));
+
+ List<Object> row2 = new ArrayList<Object>(2);
+ row2.add(new Integer(0));
+ row2.add(new Integer(100));
+
+ helpTestGetStaticSize(new List[] { row1, row2 }, 144);
+ }
+
+ @Test public void testGetSizeBigInteger() {
+ BigInteger b = BigInteger.ONE;
+
+ helpTestGetStaticSize(b, 48);
+ }
+
+ @Test public void testGetSizeBigDecimal() {
+ BigDecimal bd = new BigDecimal("1.0"); //$NON-NLS-1$
+
+ helpTestGetStaticSize(bd, 96);
+ }
+
+ @Test public void testGetSizeByteArray() {
+ byte[] bytes = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10};
+ helpTestGetSize(bytes, 32);
+ }
+
+ @Test public void testResultSet() {
+ List<?>[] expected = new List[] {
+ Arrays.asList(new Object[] { "a", new Integer(0),
Boolean.FALSE, new Double(2.0), "a", new Integer(0) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "a", new Integer(0),
Boolean.FALSE, new Double(2.0), "a", new Integer(0) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "a", new Integer(0),
Boolean.FALSE, new Double(2.0), "a", new Integer(0) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "a", new Integer(0),
Boolean.FALSE, new Double(2.0), "a", new Integer(0) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "a", new Integer(0),
Boolean.FALSE, new Double(2.0), "a", new Integer(3) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "a", new Integer(0),
Boolean.FALSE, new Double(2.0), "a", new Integer(3) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "a", new Integer(3),
Boolean.TRUE, new Double(7.0), "a", new Integer(0) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "a", new Integer(3),
Boolean.TRUE, new Double(7.0), "a", new Integer(0) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "a", new Integer(3),
Boolean.TRUE, new Double(7.0), "a", new Integer(3) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "b", new Integer(2),
Boolean.FALSE, new Double(0.0), "b", new Integer(2) }), //$NON-NLS-1$
//$NON-NLS-2$
+ Arrays.asList(new Object[] { "c", new Integer(1),
Boolean.FALSE, new Double(0.0), "c", new Integer(1) }) //$NON-NLS-1$
//$NON-NLS-2$
+ };
+
+ String[] types = {"string", "integer", "boolean",
"double", "string", "integer"};
//$NON-NLS-1$//$NON-NLS-2$//$NON-NLS-3$//$NON-NLS-4$ //$NON-NLS-5$//$NON-NLS-6$
+
+ TupleBatch tb = new TupleBatch(1, expected);
+ tb.setDataTypes(types);
+ long actualSize = new SizeUtility().getBatchSize(tb);
+ assertEquals("Got unexpected size: ", 2667, actualSize); //$NON-NLS-1$
+ }
+
+}
\ No newline at end of file
Property changes on:
branches/7.4.x/engine/src/test/java/org/teiid/common/buffer/impl/TestSizeUtility.java
___________________________________________________________________
Added: svn:mime-type
+ text/plain
Modified:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestAggregateProcessing.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestAggregateProcessing.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestAggregateProcessing.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -31,6 +31,8 @@
import java.util.List;
import org.junit.Test;
+import org.teiid.common.buffer.BufferManagerFactory;
+import org.teiid.common.buffer.impl.BufferManagerImpl;
import org.teiid.query.metadata.QueryMetadataInterface;
import org.teiid.query.optimizer.TestAggregatePushdown;
import org.teiid.query.optimizer.TestOptimizer;
@@ -38,6 +40,7 @@
import org.teiid.query.optimizer.capabilities.FakeCapabilitiesFinder;
import org.teiid.query.sql.lang.Command;
import org.teiid.query.unittest.RealMetadataFactory;
+import org.teiid.query.util.CommandContext;
import org.teiid.translator.SourceSystemFunctions;
@SuppressWarnings({"nls", "unchecked"})
@@ -372,5 +375,29 @@
// Run query
helpProcess(plan, dataManager, expected);
}
+
+ @Test public void testArrayAggOrderByPersistence() throws Exception {
+ // Create query
+ String sql = "SELECT array_agg(e2 order by e1) from pm1.g1 group by e3";
//$NON-NLS-1$
+ // Create expected results
+ List[] expected = new List[] {
+ Arrays.asList((Object)new Integer[] {1, 0, 0, 2}),
+ Arrays.asList((Object)new Integer[] {3, 1}),
+ };
+
+ // Construct data manager with data
+ FakeDataManager dataManager = new FakeDataManager();
+ sampleData1(dataManager);
+
+ // Plan query
+ ProcessorPlan plan = helpGetPlan(sql, RealMetadataFactory.example1Cached());
+ CommandContext cc = TestProcessor.createCommandContext();
+ BufferManagerImpl impl = BufferManagerFactory.getTestBufferManager(0, 2);
+ impl.setUseWeakReferences(false);
+ cc.setBufferManager(impl);
+ // Run query
+ helpProcess(plan, cc, dataManager, expected);
+ }
+
}
Modified:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestProcessor.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestProcessor.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/TestProcessor.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -320,7 +320,7 @@
List record = ts.nextTuple();
//handle xml
- if(record.size() == 1){
+ if(record.size() == 1 && expectedResults[i].size() == 1){
Object cellValue = record.get(0);
if(cellValue instanceof XMLType){
XMLType id = (XMLType)cellValue;
@@ -329,6 +329,9 @@
compareDocuments((String)expectedResults[i].get(0), actualDoc);
continue;
}
+ } else if (cellValue instanceof Object[]) {
+ assertArrayEquals((Object[])expectedResults[i].get(0),
(Object[])cellValue);
+ continue;
}
}
Deleted:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/NodeTestUtil.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/NodeTestUtil.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/NodeTestUtil.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -1,73 +0,0 @@
-/*
- * 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.query.processor.relational;
-
-import org.teiid.common.buffer.BufferManager;
-import org.teiid.common.buffer.StorageManager;
-import org.teiid.common.buffer.impl.BufferManagerImpl;
-import org.teiid.common.buffer.impl.MemoryStorageManager;
-import org.teiid.core.TeiidComponentException;
-
-/**
- * @since 4.2
- */
-public class NodeTestUtil {
-
- static BufferManager getTestBufferManager(long bytesAvailable, int procBatchSize, int
connectorBatchSize) {
- BufferManagerImpl bufferManager = new BufferManagerImpl();
- bufferManager.setProcessorBatchSize(procBatchSize);
- bufferManager.setConnectorBatchSize(connectorBatchSize);
-
bufferManager.setMaxProcessingBatchColumns((int)(bytesAvailable/procBatchSize/BufferManagerImpl.KB_PER_VALUE/1024));
-
bufferManager.setMaxReserveBatchColumns((int)(bytesAvailable/procBatchSize/BufferManagerImpl.KB_PER_VALUE/1024));
- // Get the properties for BufferManager
- return createBufferManager(bufferManager);
- }
-
- static BufferManager getTestBufferManager(long bytesAvailable, int procBatchSize) {
- BufferManagerImpl bufferManager = new BufferManagerImpl();
- bufferManager.setProcessorBatchSize(procBatchSize);
-
bufferManager.setMaxProcessingBatchColumns((int)(bytesAvailable/procBatchSize/BufferManagerImpl.KB_PER_VALUE/1024));
-
bufferManager.setMaxReserveBatchColumns((int)(bytesAvailable/procBatchSize/BufferManagerImpl.KB_PER_VALUE/1024));
- // Get the properties for BufferManager
- return createBufferManager(bufferManager);
- }
-
- static BufferManager createBufferManager(BufferManagerImpl bufferManager) {
- try {
- bufferManager.initialize();
- } catch (TeiidComponentException e) {
- throw new RuntimeException(e);
- }
-
- // Add storage managers
-
- bufferManager.setStorageManager(createFakeDatabaseStorageManager());
- return bufferManager;
- }
-
-
- private static StorageManager createFakeDatabaseStorageManager() {
- return new MemoryStorageManager();
- }
-
-}
Modified:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestGroupingNode.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestGroupingNode.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestGroupingNode.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -53,7 +53,7 @@
import org.teiid.query.unittest.RealMetadataFactory;
import org.teiid.query.util.CommandContext;
-
+@SuppressWarnings("unchecked")
public class TestGroupingNode {
public static FakeTupleSource createTupleSource1() {
Modified:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestJoinNode.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestJoinNode.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestJoinNode.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -34,6 +34,7 @@
import org.junit.Test;
import org.teiid.common.buffer.BlockedException;
import org.teiid.common.buffer.BufferManager;
+import org.teiid.common.buffer.BufferManagerFactory;
import org.teiid.common.buffer.TupleBatch;
import org.teiid.core.TeiidComponentException;
import org.teiid.core.TeiidProcessingException;
@@ -231,7 +232,7 @@
}
public void helpTestJoinDirect(List[] expectedResults, int batchSize, int
processingBytes) throws TeiidComponentException, TeiidProcessingException {
- BufferManager mgr = NodeTestUtil.getTestBufferManager(processingBytes,
batchSize);
+ BufferManager mgr = BufferManagerFactory.getTestBufferManager(processingBytes,
batchSize);
CommandContext context = new CommandContext("pid", "test",
null, null, 1); //$NON-NLS-1$ //$NON-NLS-2$
join.addChild(leftNode);
Modified:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestProjectIntoNode.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestProjectIntoNode.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestProjectIntoNode.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -32,6 +32,7 @@
import org.teiid.api.exception.query.ExpressionEvaluationException;
import org.teiid.common.buffer.BlockedException;
import org.teiid.common.buffer.BufferManager;
+import org.teiid.common.buffer.BufferManagerFactory;
import org.teiid.common.buffer.TupleBatch;
import org.teiid.common.buffer.TupleSource;
import org.teiid.core.TeiidComponentException;
@@ -80,7 +81,7 @@
CommandContext context = new CommandContext();
context.setProcessorID("processorID"); //$NON-NLS-1$
- BufferManager bm = NodeTestUtil.getTestBufferManager(tupleBatchSize,
tupleBatchSize);
+ BufferManager bm = BufferManagerFactory.getTestBufferManager(tupleBatchSize,
tupleBatchSize);
ProcessorDataManager dataManager = new FakePDM(tupleBatchSize);
child.initialize(context, bm, dataManager);
Modified:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestSortNode.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestSortNode.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestSortNode.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -52,7 +52,7 @@
public static final int BATCH_SIZE = 100;
private void helpTestSort(List elements, List[] data, List sortElements, List
sortTypes, List[] expected, Mode mode) throws TeiidComponentException,
TeiidProcessingException {
- BufferManager mgr = NodeTestUtil.getTestBufferManager(100, BATCH_SIZE,
BATCH_SIZE);
+ BufferManager mgr = BufferManagerFactory.getTestBufferManager(100, BATCH_SIZE,
BATCH_SIZE);
CommandContext context = new CommandContext ("pid", "test",
null, null, 1); //$NON-NLS-1$ //$NON-NLS-2$
BlockingFakeRelationalNode dataNode = new BlockingFakeRelationalNode(2, data);
Modified:
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestUnionAllNode.java
===================================================================
---
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestUnionAllNode.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/engine/src/test/java/org/teiid/query/processor/relational/TestUnionAllNode.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -31,6 +31,7 @@
import org.junit.Test;
import org.teiid.common.buffer.BlockedException;
import org.teiid.common.buffer.BufferManager;
+import org.teiid.common.buffer.BufferManagerFactory;
import org.teiid.common.buffer.TupleBatch;
import org.teiid.core.TeiidComponentException;
import org.teiid.core.TeiidProcessingException;
@@ -45,7 +46,7 @@
public class TestUnionAllNode {
public void helpTestUnion(RelationalNode[] children, RelationalNode union, List[]
expected) throws TeiidComponentException, TeiidProcessingException {
- BufferManager mgr = NodeTestUtil.getTestBufferManager(1, 2);
+ BufferManager mgr = BufferManagerFactory.getTestBufferManager(1, 2);
CommandContext context = new CommandContext("pid", "test",
null, null, 1); //$NON-NLS-1$ //$NON-NLS-2$
FakeDataManager fdm = new FakeDataManager();
for(int i=0; i<children.length; i++) {
Modified:
branches/7.4.x/runtime/src/main/java/org/teiid/deployers/PgCatalogMetadataStore.java
===================================================================
---
branches/7.4.x/runtime/src/main/java/org/teiid/deployers/PgCatalogMetadataStore.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/runtime/src/main/java/org/teiid/deployers/PgCatalogMetadataStore.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -312,10 +312,10 @@
String transformation = "SELECT t1.OID as oid, t1.Name as proname, (SELECT (CASE
WHEN count(pp.Type)>0 THEN true else false END) as x FROM ProcedureParams pp WHERE
pp.ProcedureName = t1.Name AND pp.SchemaName = t1.SchemaName and
pp.Type='ResultSet') as proretset, " + //$NON-NLS-1$
"CASE WHEN (SELECT count(dt.oid) FROM ProcedureParams pp, matpg_datatype dt WHERE
pp.ProcedureName = t1.Name AND pp.SchemaName = t1.SchemaName AND pp.Type IN
('ReturnValue', 'ResultSet') AND dt.Name = pp.DataType) IS NULL THEN
(select oid from pg_type WHERE typname = 'void') WHEN (SELECT count(dt.oid) FROM
ProcedureParams pp, matpg_datatype dt WHERE pp.ProcedureName = t1.Name AND pp.SchemaName =
t1.SchemaName AND pp.Type = 'ResultSet' AND dt.Name = pp.DataType) IS NOT NULL
THEN (select oid from pg_type WHERE typname = 'record') ELSE (SELECT dt.oid FROM
ProcedureParams pp, matpg_datatype dt WHERE pp.ProcedureName = t1.Name AND pp.SchemaName =
t1.SchemaName AND pp.Type = 'ReturnValue' AND dt.Name = pp.DataType) END as
prorettype, " + //$NON-NLS-1$
"convert((SELECT count(*) FROM ProcedureParams pp WHERE pp.ProcedureName = t1.Name
AND pp.SchemaName = t1.SchemaName AND pp.Type IN ('In', 'InOut')), short)
as pronargs, " + //$NON-NLS-1$
- "(select "+textAggStmt("y.oid","y.type, y.position"
)+" FROM ("+paramTable("'ResultSet','ReturnValue',
'Out'")+") as y) as proargtypes, " +//$NON-NLS-1$ //$NON-NLS-2$
//$NON-NLS-3$
- "(select "+textAggStmt("y.name", "y.type,
y.position")+" FROM (SELECT pp.Name as name, pp.position as position, pp.Type as
type FROM ProcedureParams pp WHERE pp.ProcedureName = t1.Name AND pp.SchemaName =
t1.SchemaName AND pp.Type NOT IN ('ReturnValue' )) as y) as proargnames, " +
//$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
- "(select case WHEN count(distinct(y.type)) = 1 THEN null ELSE
"+textAggStmt("CASE WHEN (y.type ='In') THEN 'i' WHEN (y.type =
'Out') THEN 'o' WHEN (y.type = 'InOut') THEN 'b' WHEN
(y.type = 'ResultSet') THEN 't' END",
"y.type,y.position")+" END FROM (SELECT pp.Type as type, pp.Position as
position FROM ProcedureParams pp WHERE pp.ProcedureName = t1.Name AND pp.SchemaName =
t1.SchemaName AND pp.Type NOT IN ('ReturnValue')) as y) as proargmodes, " +
//$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
- "(select case WHEN count(distinct(y.oid)) = 1 THEN null ELSE
"+textAggStmt("y.oid", "y.type, y.position")+" END FROM
("+paramTable("'ReturnValue'")+") as y) as proallargtypes,
" + //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
+ "(select "+arrayAgg("y.oid","y.type, y.position" )+"
FROM ("+paramTable("'ResultSet','ReturnValue',
'Out'")+") as y) as proargtypes, " +//$NON-NLS-1$ //$NON-NLS-2$
//$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$ //$NON-NLS-6$
+ "(select "+arrayAgg("y.name", "y.type,
y.position")+" FROM (SELECT pp.Name as name, pp.position as position, pp.Type as
type FROM ProcedureParams pp WHERE pp.ProcedureName = t1.Name AND pp.SchemaName =
t1.SchemaName AND pp.Type NOT IN ('ReturnValue' )) as y) as proargnames, " +
//$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$
+ "(select case WHEN count(distinct(y.type)) = 1 THEN null ELSE
"+arrayAgg("CASE WHEN (y.type ='In') THEN 'i' WHEN (y.type =
'Out') THEN 'o' WHEN (y.type = 'InOut') THEN 'b' WHEN
(y.type = 'ResultSet') THEN 't' END",
"y.type,y.position")+" END FROM (SELECT pp.Type as type, pp.Position as
position FROM ProcedureParams pp WHERE pp.ProcedureName = t1.Name AND pp.SchemaName =
t1.SchemaName AND pp.Type NOT IN ('ReturnValue')) as y) as proargmodes, " +
//$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$
+ "(select case WHEN count(distinct(y.oid)) = 1 THEN null ELSE
"+arrayAgg("y.oid", "y.type, y.position")+" END FROM
("+paramTable("'ReturnValue'")+") as y) as proallargtypes,
" + //$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$ //$NON-NLS-4$ //$NON-NLS-5$
//$NON-NLS-6$
"(SELECT OID FROM SYS.Schemas WHERE Name = t1.SchemaName) as pronamespace " +
//$NON-NLS-1$
"FROM SYS.Procedures as t1";//$NON-NLS-1$
@@ -330,7 +330,7 @@
}
- private String textAggStmt(String select, String orderby) {
+ private String arrayAgg(String select, String orderby) {
return "array_agg("+select+" ORDER BY "+orderby+")";
//$NON-NLS-1$ //$NON-NLS-2$ //$NON-NLS-3$
}
Modified: branches/7.4.x/runtime/src/main/java/org/teiid/services/BufferServiceImpl.java
===================================================================
---
branches/7.4.x/runtime/src/main/java/org/teiid/services/BufferServiceImpl.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/runtime/src/main/java/org/teiid/services/BufferServiceImpl.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -62,8 +62,8 @@
private int connectorBatchSize = BufferManager.DEFAULT_CONNECTOR_BATCH_SIZE;
private int maxOpenFiles = FileStorageManager.DEFAULT_MAX_OPEN_FILES;
private long maxFileSize = FileStorageManager.DEFAULT_MAX_FILESIZE; // 2GB
- private int maxProcessingBatchesColumns =
BufferManager.DEFAULT_MAX_PROCESSING_BATCHES;
- private int maxReserveBatchColumns = BufferManager.DEFAULT_RESERVE_BUFFERS;
+ private int maxProcessingKb = BufferManager.DEFAULT_MAX_PROCESSING_KB;
+ private int maxReserveKb = BufferManager.DEFAULT_RESERVE_BUFFER_KB;
private long maxBufferSpace = FileStorageManager.DEFAULT_MAX_BUFFERSPACE;
private FileStorageManager fsm;
@@ -86,8 +86,8 @@
this.bufferMgr = new BufferManagerImpl();
this.bufferMgr.setConnectorBatchSize(Integer.valueOf(connectorBatchSize));
this.bufferMgr.setProcessorBatchSize(Integer.valueOf(processorBatchSize));
- this.bufferMgr.setMaxReserveBatchColumns(this.maxReserveBatchColumns);
-
this.bufferMgr.setMaxProcessingBatchColumns(this.maxProcessingBatchesColumns);
+ this.bufferMgr.setMaxReserveKB(this.maxReserveKb);
+ this.bufferMgr.setMaxProcessingKB(this.maxProcessingKb);
this.bufferMgr.initialize();
@@ -167,14 +167,6 @@
this.maxFileSize = maxFileSize;
}
- public void setMaxReserveBatchColumns(int value) {
- this.maxReserveBatchColumns = value;
- }
-
- public void setMaxProcessingBatchesColumns(int value) {
- this.maxProcessingBatchesColumns = value;
- }
-
@ManagementProperty(description="Max file size, in MB, for buffer files (default
2GB)")
public long getMaxFileSize() {
return maxFileSize;
@@ -185,20 +177,24 @@
this.maxOpenFiles = maxOpenFiles;
}
- @ManagementProperty(description="The number of batch columns guarenteed to a
processing operation. Set this value lower if the workload typically" +
- "processes larger numbers of concurrent queries with large intermediate
results from operations such as sorting, " +
- "grouping, etc. (default 128)")
- public int getMaxProcessingBatchesColumns() {
- return maxProcessingBatchesColumns;
+ @ManagementProperty(description="The approximate amount of buffer memory in
kilobytes allowable for a single processing operation (sort, grouping, etc.) regardless of
existing memory commitments. -1 means to automatically calculate a value (default
-1).")
+ public int getMaxProcessingKb() {
+ return maxProcessingKb;
}
- @ManagementProperty(description="The number of batch columns to allow in memory
(default 16384). " +
- "This value should be set lower or higher depending on the available memory to
Teiid in the VM. " +
- "16384 is considered a good default for a dedicated 32-bit VM running Teiid
with a 1 gig heap.")
- public int getMaxReserveBatchColumns() {
- return maxReserveBatchColumns;
+ @ManagementProperty(description="The approximate amount of memory in kilobytes
allowed to be held by the buffer manager. -1 means to automatically calculate a value
(default -1)")
+ public int getMaxReservedKb() {
+ return maxReserveKb;
}
+ public void setMaxProcessingKb(int maxProcessingKb) {
+ this.maxProcessingKb = maxProcessingKb;
+ }
+
+ public void setMaxReserveKb(int maxReserveKb) {
+ this.maxReserveKb = maxReserveKb;
+ }
+
@ManagementProperty(description="Max file storage space, in MB, to be used for
buffer files (default 50G)")
public long getMaxBufferSpace() {
return maxBufferSpace;
Modified:
branches/7.4.x/runtime/src/test/java/org/teiid/dqp/service/buffer/TestLocalBufferService.java
===================================================================
---
branches/7.4.x/runtime/src/test/java/org/teiid/dqp/service/buffer/TestLocalBufferService.java 2011-06-07
21:36:50 UTC (rev 3230)
+++
branches/7.4.x/runtime/src/test/java/org/teiid/dqp/service/buffer/TestLocalBufferService.java 2011-06-07
21:36:58 UTC (rev 3231)
@@ -91,7 +91,7 @@
svc.start();
BufferManager mgr = svc.getBufferManager();
- assertEquals(16261, mgr.getSchemaSize(schema));
+ assertEquals(13141, mgr.getSchemaSize(schema));
}
}