Author: shawkins
Date: 2010-02-02 12:03:37 -0500 (Tue, 02 Feb 2010)
New Revision: 1797
Modified:
trunk/engine/src/main/java/com/metamatrix/common/buffer/TupleBuffer.java
trunk/engine/src/main/java/com/metamatrix/common/buffer/impl/BufferManagerImpl.java
trunk/engine/src/main/java/com/metamatrix/query/processor/BatchIterator.java
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/JoinNode.java
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/MergeJoinStrategy.java
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/NestedLoopJoinStrategy.java
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/PartitionedSortJoin.java
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/SourceState.java
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/SubqueryAwareEvaluator.java
trunk/engine/src/test/java/com/metamatrix/query/processor/TestOptionalJoins.java
Log:
TEIID-913 updating the buffering strategy for joins to buffer on read and only buffer the
match region for merge joins.
Modified: trunk/engine/src/main/java/com/metamatrix/common/buffer/TupleBuffer.java
===================================================================
--- trunk/engine/src/main/java/com/metamatrix/common/buffer/TupleBuffer.java 2010-02-02
15:49:20 UTC (rev 1796)
+++ trunk/engine/src/main/java/com/metamatrix/common/buffer/TupleBuffer.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -129,7 +129,7 @@
@Override
public int available() {
- return 0;
+ return rowCount - currentRow + 1;
}
}
@@ -222,6 +222,16 @@
}
}
+ public void purge() {
+ if (this.batchBuffer != null) {
+ this.batchBuffer = null;
+ }
+ for (BatchManager.ManagedBatch batch : this.batches.values()) {
+ batch.remove();
+ }
+ this.batches.clear();
+ }
+
/**
* Force the persistence of any rows held in memory.
* @throws MetaMatrixComponentException
@@ -302,13 +312,7 @@
public void remove() {
if (!removed) {
this.manager.remove();
- if (this.batchBuffer != null) {
- this.batchBuffer = null;
- }
- for (BatchManager.ManagedBatch batch : this.batches.values()) {
- batch.remove();
- }
- this.batches.clear();
+ purge();
}
}
Modified:
trunk/engine/src/main/java/com/metamatrix/common/buffer/impl/BufferManagerImpl.java
===================================================================
---
trunk/engine/src/main/java/com/metamatrix/common/buffer/impl/BufferManagerImpl.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/main/java/com/metamatrix/common/buffer/impl/BufferManagerImpl.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -79,6 +79,8 @@
* for our tendency to read buffers in a forward manner. If our processing algorithms
are changed
* to use alternating ascending/descending access, then the buffering approach could be
replaced
* with a simple LRU.
+ *
+ * TODO: allow for cached stores to use lru - (result set/mat view)
*/
public class BufferManagerImpl implements BufferManager, StorageManager {
@@ -387,8 +389,7 @@
@Override
public TupleBuffer createTupleBuffer(final List elements, String groupName,
- TupleSourceType tupleSourceType)
- throws MetaMatrixComponentException {
+ TupleSourceType tupleSourceType) {
final String newID = String.valueOf(this.currentTuple.getAndIncrement());
BatchManager batchManager = new BatchManager() {
Modified: trunk/engine/src/main/java/com/metamatrix/query/processor/BatchIterator.java
===================================================================
---
trunk/engine/src/main/java/com/metamatrix/query/processor/BatchIterator.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/main/java/com/metamatrix/query/processor/BatchIterator.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -28,13 +28,23 @@
import com.metamatrix.api.exception.MetaMatrixProcessingException;
import com.metamatrix.common.buffer.IndexedTupleSource;
import com.metamatrix.common.buffer.TupleBatch;
+import com.metamatrix.common.buffer.TupleBuffer;
import com.metamatrix.query.processor.BatchCollector.BatchProducer;
import com.metamatrix.query.sql.symbol.SingleElementSymbol;
-public class BatchIterator implements
- IndexedTupleSource {
+/**
+ * A BatchIterator provides an iterator interface to a {@link BatchProducer}.
+ * By setting {@link #setBuffer(TupleBuffer)},
+ * the iterator can copy on read into a {@link TupleBuffer} for repeated reading.
+ *
+ * Note that the saveOnMark buffering only lasts until the next mark is set.
+ */
+public class BatchIterator implements IndexedTupleSource {
private final BatchProducer source;
+ private boolean saveOnMark;
+ private TupleBuffer buffer;
+ private IndexedTupleSource bufferedTs;
public BatchIterator(BatchProducer source) {
this.source = source;
@@ -44,15 +54,33 @@
private int currentRow = 1;
private TupleBatch currentBatch;
private List currentTuple;
-
+ private int bufferedIndex;
+ private boolean mark;
+
+ @Override
public boolean hasNext() throws MetaMatrixComponentException,
MetaMatrixProcessingException {
- if (done) {
+
+ if (done && this.bufferedTs == null) {
return false;
}
while (currentTuple == null) {
if (currentBatch == null) {
+ if (this.bufferedTs != null) {
+ if (this.currentRow <= this.bufferedIndex) {
+ this.bufferedTs.setPosition(currentRow++);
+ this.currentTuple = this.bufferedTs.nextTuple();
+ return true;
+ }
+ if (done) {
+ return false;
+ }
+ }
currentBatch = this.source.nextBatch();
+ if (buffer != null && !saveOnMark) {
+ buffer.addTupleBatch(currentBatch, true);
+ bufferedIndex = currentBatch.getEndRow();
+ }
}
if (currentBatch.getEndRow() >= currentRow) {
@@ -68,9 +96,18 @@
return true;
}
+ public void setBuffer(TupleBuffer buffer, boolean saveOnMark) {
+ this.buffer = buffer;
+ this.bufferedTs = this.buffer.createIndexedTupleSource();
+ this.saveOnMark = saveOnMark;
+ }
+
@Override
public void closeSource() throws MetaMatrixComponentException {
-
+ if (this.buffer != null) {
+ this.buffer.remove();
+ this.buffer = null;
+ }
}
@Override
@@ -86,15 +123,32 @@
}
List result = currentTuple;
currentTuple = null;
+ if (mark && saveOnMark && this.currentRow - 1 >
this.buffer.getRowCount()) {
+ this.buffer.addTupleBatch(new TupleBatch(this.currentRow - 1, new List[]
{result}), true);
+ this.bufferedIndex = this.currentRow - 1;
+ }
return result;
}
public void reset() {
+ if (this.bufferedTs != null) {
+ mark = false;
+ this.bufferedTs.reset();
+ this.currentRow = this.bufferedTs.getCurrentIndex();
+ return;
+ }
throw new UnsupportedOperationException();
}
public void mark() {
- //does nothing
+ if (this.bufferedTs != null) {
+ this.bufferedTs.mark();
+ if (saveOnMark && this.currentRow > this.bufferedIndex) {
+ this.buffer.purge();
+ this.bufferedIndex = 0;
+ }
+ }
+ mark = true;
}
@Override
@@ -103,21 +157,26 @@
}
public void setPosition(int position) {
- if (position == this.currentRow) {
- return;
+ if (this.bufferedTs != null) {
+ this.bufferedTs.setPosition(position);
+ this.currentRow = position;
}
- if (position < this.currentRow && (this.currentBatch == null || position
< this.currentBatch.getBeginRow())) {
+ if (this.currentBatch == null && position < this.currentRow) {
throw new UnsupportedOperationException("Backwards positioning is not
allowed"); //$NON-NLS-1$
- }
- this.currentRow = position;
+ }
+ this.currentRow = position;
this.currentTuple = null;
- if (currentBatch.getEndRow() < currentRow) {
+ if (this.currentBatch == null || !this.currentBatch.containsRow(position)) {
this.currentBatch = null;
- }
+ }
}
@Override
public int available() {
+ if (this.currentRow <= this.bufferedIndex) {
+ this.bufferedTs.setPosition(this.currentRow);
+ return this.bufferedTs.available();
+ }
if (currentBatch != null) {
return currentBatch.getEndRow() - currentRow + 1;
}
Modified:
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/JoinNode.java
===================================================================
---
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/JoinNode.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/JoinNode.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -37,6 +37,7 @@
import com.metamatrix.common.buffer.TupleBatch;
import com.metamatrix.common.buffer.TupleBuffer;
import com.metamatrix.query.processor.ProcessorDataManager;
+import com.metamatrix.query.processor.relational.SourceState.ImplicitBuffer;
import com.metamatrix.query.sql.LanguageObject;
import com.metamatrix.query.sql.lang.Criteria;
import com.metamatrix.query.sql.lang.JoinType;
@@ -188,7 +189,10 @@
MetaMatrixComponentException,
MetaMatrixProcessingException {
if (state == State.LOAD_LEFT) {
- //left child was already opened by the join node
+ if (this.joinType != JoinType.JOIN_FULL_OUTER) {
+ this.joinStrategy.leftSource.setImplicitBuffer(ImplicitBuffer.NONE);
+ }
+ //left child was already opened by the join node
this.joinStrategy.loadLeft();
state = State.LOAD_RIGHT;
}
@@ -201,11 +205,6 @@
this.rightOpened = true;
}
this.joinStrategy.loadRight();
- //force buffering based upon join type - may have already happened in the
strategy load methods
- this.joinStrategy.rightSource.getTupleBuffer();
- if (joinType == JoinType.JOIN_FULL_OUTER) {
- this.joinStrategy.leftSource.getTupleBuffer();
- }
state = State.EXECUTE;
}
Modified:
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/MergeJoinStrategy.java
===================================================================
---
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/MergeJoinStrategy.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/MergeJoinStrategy.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -22,15 +22,13 @@
package com.metamatrix.query.processor.relational;
-import java.util.Collections;
import java.util.List;
import com.metamatrix.api.exception.MetaMatrixComponentException;
import com.metamatrix.api.exception.MetaMatrixProcessingException;
import com.metamatrix.api.exception.query.CriteriaEvaluationException;
-import com.metamatrix.query.processor.relational.SortUtility.Mode;
+import com.metamatrix.query.processor.relational.SourceState.ImplicitBuffer;
import com.metamatrix.query.sql.lang.JoinType;
-import com.metamatrix.query.sql.lang.OrderBy;
/**
* MergeJoinStrategy supports generalized Full, Left Outer, and Inner Joins (containing
non-equi join criteria) as long as there
@@ -85,8 +83,6 @@
private boolean grouping;
//load time state
- private SortUtility leftSort;
- private SortUtility rightSort;
protected SortOption processingSortLeft;
protected SortOption processingSortRight;
@@ -136,8 +132,6 @@
super.close();
this.outerState = null;
this.innerState = null;
- this.leftSort = null;
- this.rightSort = null;
}
@Override
@@ -331,33 +325,16 @@
@Override
protected void loadLeft() throws MetaMatrixComponentException,
MetaMatrixProcessingException {
- if (this.processingSortLeft == SortOption.SORT || this.processingSortLeft ==
SortOption.SORT_DISTINCT) {
- if (this.leftSort == null) {
- List expressions = this.joinNode.getLeftExpressions();
- this.leftSort = new SortUtility(this.leftSource.getIterator(),
- expressions,
Collections.nCopies(expressions.size(), OrderBy.ASC), processingSortLeft ==
SortOption.SORT_DISTINCT?Mode.DUP_REMOVE_SORT:Mode.SORT,
- this.joinNode.getBufferManager(),
this.joinNode.getConnectionID());
- this.leftSource.markDistinct(processingSortLeft ==
SortOption.SORT_DISTINCT && expressions.size() ==
this.leftSource.getOuterVals().size());
- }
- this.leftSource.setTupleSource(leftSort.sort());
- this.leftSource.markDistinct(leftSort.isDistinct());
- }
+ this.leftSource.sort(this.processingSortLeft);
}
@Override
protected void loadRight() throws MetaMatrixComponentException,
MetaMatrixProcessingException {
- if (this.processingSortRight == SortOption.SORT || this.processingSortRight ==
SortOption.SORT_DISTINCT) {
- if (this.rightSort == null) {
- List expressions = this.joinNode.getRightExpressions();
- this.rightSort = new SortUtility(this.rightSource.getIterator(),
- expressions,
Collections.nCopies(expressions.size(), OrderBy.ASC), processingSortRight ==
SortOption.SORT_DISTINCT?Mode.DUP_REMOVE_SORT:Mode.SORT,
- this.joinNode.getBufferManager(),
this.joinNode.getConnectionID());
- this.rightSource.markDistinct(processingSortRight == SortOption.SORT_DISTINCT
&& expressions.size() == this.rightSource.getOuterVals().size());
- }
- this.rightSource.setTupleSource(rightSort.sort());
- this.rightSource.markDistinct(rightSort.isDistinct());
- }
+ this.rightSource.sort(this.processingSortRight);
+ if (this.joinNode.getJoinType() != JoinType.JOIN_FULL_OUTER) {
+ this.rightSource.setImplicitBuffer(ImplicitBuffer.ON_MARK);
+ }
}
public void setProcessingSortRight(boolean processingSortRight) {
Modified:
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/NestedLoopJoinStrategy.java
===================================================================
---
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/NestedLoopJoinStrategy.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/NestedLoopJoinStrategy.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -24,6 +24,10 @@
import java.util.List;
+import com.metamatrix.api.exception.MetaMatrixComponentException;
+import com.metamatrix.api.exception.MetaMatrixProcessingException;
+import com.metamatrix.query.processor.relational.SourceState.ImplicitBuffer;
+
/**
* Nested loop is currently implemented as a degenerate case of merge join.
*
@@ -55,6 +59,12 @@
return 0; // there are no expressions in nested loop joins, comparison is
meaningless
}
+ @Override
+ protected void loadRight() throws MetaMatrixComponentException,
+ MetaMatrixProcessingException {
+ this.rightSource.setImplicitBuffer(ImplicitBuffer.FULL);
+ }
+
/**
* @see com.metamatrix.query.processor.relational.MergeJoinStrategy#toString()
*/
Modified:
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/PartitionedSortJoin.java
===================================================================
---
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/PartitionedSortJoin.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/PartitionedSortJoin.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -202,9 +202,8 @@
this.partitions.get(index).addTuple(tuple);
}
for (TupleBuffer partition : this.partitions) {
- partition.saveBatch();
+ partition.close();
}
- this.partitionedSource.getIterator().setPosition(1);
}
partitioned = true;
}
@@ -220,9 +219,6 @@
}
while (currentPartition < partitions.size()) {
if (currentSource == null) {
- if (!this.partitions.isEmpty()) {
- this.partitions.get(currentPartition).close();
- }
currentSource = partitions.get(currentPartition).createIndexedTupleSource();
}
Modified:
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/SourceState.java
===================================================================
---
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/SourceState.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/SourceState.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -32,10 +32,18 @@
import com.metamatrix.common.buffer.BufferManager.TupleSourceType;
import com.metamatrix.query.processor.BatchCollector;
import com.metamatrix.query.processor.BatchIterator;
+import com.metamatrix.query.processor.relational.MergeJoinStrategy.SortOption;
+import com.metamatrix.query.processor.relational.SortUtility.Mode;
+import com.metamatrix.query.sql.lang.OrderBy;
class SourceState {
+ enum ImplicitBuffer {
+ NONE, FULL, ON_MARK
+ }
+
private RelationalNode source;
+ private List expressions;
private BatchCollector collector;
private TupleBuffer buffer;
private List<Object> outerVals;
@@ -44,16 +52,22 @@
private List currentTuple;
private int maxProbeMatch = 1;
private boolean distinct;
+ private ImplicitBuffer implicitBuffer = ImplicitBuffer.FULL;
- private boolean canBuffer = true;
+ private SortUtility sortUtility;
public SourceState(RelationalNode source, List expressions) {
this.source = source;
+ this.expressions = expressions;
List elements = source.getElements();
this.outerVals = Collections.nCopies(elements.size(), null);
this.expressionIndexes = getExpressionIndecies(expressions, elements);
}
+ public void setImplicitBuffer(ImplicitBuffer implicitBuffer) {
+ this.implicitBuffer = implicitBuffer;
+ }
+
private int[] getExpressionIndecies(List expressions,
List elements) {
if (expressions == null) {
@@ -75,43 +89,41 @@
return currentTuple;
}
- public void reset() {
+ public void reset() throws MetaMatrixComponentException {
this.getIterator().reset();
this.getIterator().mark();
this.currentTuple = null;
}
public void close() {
- closeTupleSource();
- }
-
- private void closeTupleSource() {
- if (this.buffer != null) {
+ if (this.buffer != null) {
this.buffer.remove();
this.buffer = null;
}
if (this.iterator != null) {
+ try {
+ this.iterator.closeSource();
+ } catch (MetaMatrixComponentException e) {
+ }
this.iterator = null;
}
}
-
+
public int getRowCount() throws MetaMatrixComponentException,
MetaMatrixProcessingException {
return this.getTupleBuffer().getRowCount();
}
- void setTupleSource(TupleBuffer result) {
- closeTupleSource();
- this.buffer = result;
- }
-
- IndexedTupleSource getIterator() {
+ IndexedTupleSource getIterator() throws MetaMatrixComponentException {
if (this.iterator == null) {
if (this.buffer != null) {
iterator = buffer.createIndexedTupleSource();
} else {
- canBuffer = false;
// return a TupleBatch tuplesource iterator
- iterator = new BatchIterator(this.source);
+ BatchIterator bi = new BatchIterator(this.source);
+ if (implicitBuffer != ImplicitBuffer.NONE) {
+ bi.setBuffer(createSourceTupleBuffer(), implicitBuffer ==
ImplicitBuffer.ON_MARK);
+ }
+ this.iterator = bi;
}
}
return this.iterator;
@@ -139,14 +151,13 @@
public TupleBuffer getTupleBuffer() throws MetaMatrixComponentException,
MetaMatrixProcessingException {
if (this.buffer == null) {
- if (!canBuffer) {
+ if (this.iterator instanceof BatchIterator) {
throw new AssertionError("cannot buffer the source"); //$NON-NLS-1$
}
if (collector == null) {
collector = new BatchCollector(source, createSourceTupleBuffer());
}
- TupleBuffer result = collector.collectTuples();
- setTupleSource(result);
+ this.buffer = collector.collectTuples();
}
return this.buffer;
}
@@ -158,5 +169,18 @@
public void markDistinct(boolean distinct) {
this.distinct |= distinct;
}
+
+ public void sort(SortOption sortOption) throws MetaMatrixComponentException,
MetaMatrixProcessingException {
+ if (sortOption == SortOption.SORT || sortOption == SortOption.SORT_DISTINCT) {
+ if (this.sortUtility == null) {
+ this.sortUtility = new SortUtility(this.buffer != null ?
this.buffer.createIndexedTupleSource() : new BatchIterator(this.source),
+ expressions,
Collections.nCopies(expressions.size(), OrderBy.ASC), sortOption ==
SortOption.SORT_DISTINCT?Mode.DUP_REMOVE_SORT:Mode.SORT,
+ this.source.getBufferManager(),
this.source.getConnectionID());
+ this.markDistinct(sortOption == SortOption.SORT_DISTINCT &&
expressions.size() == this.getOuterVals().size());
+ }
+ this.buffer = sortUtility.sort();
+ this.markDistinct(sortUtility.isDistinct());
+ }
+ }
}
Modified:
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/SubqueryAwareEvaluator.java
===================================================================
---
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/SubqueryAwareEvaluator.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/main/java/com/metamatrix/query/processor/relational/SubqueryAwareEvaluator.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -61,7 +61,7 @@
return;
}
processor.closeProcessing();
- collector.getTupleBuffer().close();
+ collector.getTupleBuffer().remove();
processor = null;
this.done = false;
}
@@ -100,7 +100,7 @@
SubqueryState state = this.subqueries.get(key);
if (state == null) {
state = new SubqueryState();
- state.plan = (ProcessorPlan)container.getCommand().getProcessorPlan().clone();
+ state.plan = container.getCommand().getProcessorPlan().clone();
this.subqueries.put(key, state);
}
if ((tuple == null && state.tuple != null) || (tuple != null &&
!tuple.equals(state.tuple))) {
Modified:
trunk/engine/src/test/java/com/metamatrix/query/processor/TestOptionalJoins.java
===================================================================
---
trunk/engine/src/test/java/com/metamatrix/query/processor/TestOptionalJoins.java 2010-02-02
15:49:20 UTC (rev 1796)
+++
trunk/engine/src/test/java/com/metamatrix/query/processor/TestOptionalJoins.java 2010-02-02
17:03:37 UTC (rev 1797)
@@ -25,16 +25,13 @@
import java.util.Arrays;
import java.util.List;
-import com.metamatrix.query.processor.FakeDataManager;
-import com.metamatrix.query.processor.ProcessorPlan;
-import com.metamatrix.query.processor.TestProcessor;
+import org.junit.Test;
+
import com.metamatrix.query.unittest.FakeMetadataFactory;
-import junit.framework.TestCase;
-
-public class TestOptionalJoins extends TestCase {
+public class TestOptionalJoins {
- public void testOptionalJoinNode1() {
+ @Test public void testOptionalJoinNode1() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM pm1.g1, /* optional */ pm1.g2 where
pm1.g1.e1 = 'a'"; //$NON-NLS-1$
@@ -56,7 +53,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode2() {
+ @Test public void testOptionalJoinNode2() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM pm1.g1, /* optional */ pm1.g2, pm1.g3
where pm1.g1.e1 = 'a' and pm1.g1.e1 = pm1.g3.e1"; //$NON-NLS-1$
@@ -84,7 +81,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode3() {
+ @Test public void testOptionalJoinNode3() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM pm1.g1 LEFT OUTER JOIN /* optional */
pm1.g2 on pm1.g1.e1 = pm1.g2.e1"; //$NON-NLS-1$
@@ -109,7 +106,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode4() {
+ @Test public void testOptionalJoinNode4() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM (pm1.g1 LEFT OUTER JOIN /* optional */
pm1.g2 on pm1.g1.e1 = pm1.g2.e1) LEFT OUTER JOIN /* optional */ pm1.g3 on pm1.g1.e1 =
pm1.g3.e1"; //$NON-NLS-1$
@@ -134,7 +131,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode5() {
+ @Test public void testOptionalJoinNode5() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM (pm1.g1 LEFT OUTER JOIN pm1.g2 on
pm1.g1.e1 = pm1.g2.e1) LEFT OUTER JOIN /* optional */ pm1.g3 on pm1.g1.e1 =
pm1.g3.e1"; //$NON-NLS-1$
@@ -165,7 +162,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode6() {
+ @Test public void testOptionalJoinNode6() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM (pm1.g1 LEFT OUTER JOIN /* optional */
pm1.g2 on pm1.g1.e1 = pm1.g2.e1) LEFT OUTER JOIN pm1.g3 on pm1.g1.e1 = pm1.g3.e1";
//$NON-NLS-1$
@@ -196,7 +193,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode7() {
+ @Test public void testOptionalJoinNode7() {
// Create query
String sql = "SELECT pm1.g3.e1 FROM /* optional */ (pm1.g1 LEFT OUTER JOIN
pm1.g2 on pm1.g1.e1 = pm1.g2.e1) LEFT OUTER JOIN pm1.g3 on pm1.g1.e1 = pm1.g3.e1";
//$NON-NLS-1$
@@ -221,7 +218,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode8() {
+ @Test public void testOptionalJoinNode8() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM pm1.g1 LEFT OUTER JOIN /* optional */
(select * from pm1.g2) as X on pm1.g1.e1 = x.e1"; //$NON-NLS-1$
@@ -246,7 +243,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode9() {
+ @Test public void testOptionalJoinNode9() {
// Create query
String sql = "SELECT pm1.g2.e1 FROM pm1.g2, /* optional */ vm1.g1";
//$NON-NLS-1$
@@ -271,7 +268,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode10() {
+ @Test public void testOptionalJoinNode10() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM /* optional */ vm1.g1, pm1.g1";
//$NON-NLS-1$
@@ -296,7 +293,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode11() {
+ @Test public void testOptionalJoinNode11() {
// Create query
String sql = "SELECT pm1.g1.e1 FROM pm1.g1 LEFT OUTER JOIN /* optional */
vm1.g2 on pm1.g1.e1 = vm1.g2.e1"; //$NON-NLS-1$
@@ -321,7 +318,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode12() {
+ @Test public void testOptionalJoinNode12() {
// Create query
String sql = "SELECT pm1.g3.e1 FROM /* optional */ (pm1.g1 LEFT OUTER JOIN
vm1.g1 on pm1.g1.e1 = vm1.g1.e1) LEFT OUTER JOIN pm1.g3 on pm1.g1.e1 = pm1.g3.e1";
//$NON-NLS-1$
@@ -346,7 +343,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode13() {
+ @Test public void testOptionalJoinNode13() {
// Create query
String sql = "SELECT count(pm1.g1.e1) FROM pm1.g1 LEFT OUTER JOIN /*
optional */ pm1.g2 on pm1.g1.e1 = pm1.g2.e1"; //$NON-NLS-1$
@@ -366,7 +363,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode15() {
+ @Test public void testOptionalJoinNode15() {
// Create query
String sql = "SELECT x.e1 FROM (select vm1.g1.e1, vm1.g2.e2 from vm1.g1 LEFT
OUTER JOIN /* optional */vm1.g2 on vm1.g1.e2 = vm1.g2.e2) AS x"; //$NON-NLS-1$
@@ -391,7 +388,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode16() {
+ @Test public void testOptionalJoinNode16() {
// Create query
String sql = "SELECT x.e1 FROM (select vm1.g1.e1, vm1.g2.e2 from vm1.g1 LEFT
OUTER JOIN /* optional */vm1.g2 on vm1.g1.e2 = vm1.g2.e2) AS x order by x.e1";
//$NON-NLS-1$
@@ -416,7 +413,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode17() {
+ @Test public void testOptionalJoinNode17() {
// Create query
String sql = "SELECT length(z) FROM /* optional */ pm1.g1, (select distinct
e2 as y, e3 || 'x' as z from pm1.g1 ORDER BY y, z) AS x"; //$NON-NLS-1$
@@ -440,7 +437,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode18() {
+ @Test public void testOptionalJoinNode18() {
// Create query
String sql = "SELECT x.e1 FROM (select vm1.g1.e1, vm1.g2.e2 from vm1.g1 LEFT
OUTER JOIN /* optional */vm1.g2 on vm1.g1.e2 = vm1.g2.e2) AS x"; //$NON-NLS-1$
@@ -465,7 +462,7 @@
TestProcessor.helpProcess(plan, dataManager, expected);
}
- public void testOptionalJoinNode19() {
+ @Test public void testOptionalJoinNode19() {
// Create query
String sql = "SELECT length(z) FROM /* optional */ pm1.g1 inner join (select
e2 as y, e3 || 'x' as z from pm1.g1 ORDER BY z) AS x on pm1.g1.e2=x.y";
//$NON-NLS-1$