You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2015/09/08 19:10:33 UTC

[01/20] jena git commit: JENA-1023: Initial code for join algorithms.

Repository: jena
Updated Branches:
  refs/heads/master 9cfff3a3e -> 668db14a4


JENA-1023: Initial code for join algorithms.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/9f158b67
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/9f158b67
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/9f158b67

Branch: refs/heads/master
Commit: 9f158b675ec20875546389654711e60e4006cd4f
Parents: 9cfff3a
Author: Andy Seaborne <an...@apache.org>
Authored: Mon Sep 7 19:47:17 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Mon Sep 7 19:47:17 2015 +0100

----------------------------------------------------------------------
 .../jena/sparql/engine/join/HashProbeTable.java | 108 +++++++++
 .../apache/jena/sparql/engine/join/Join.java    |  77 ++++++
 .../apache/jena/sparql/engine/join/JoinKey.java | 108 +++++++++
 .../apache/jena/sparql/engine/join/JoinLib.java |  65 +++++
 .../sparql/engine/join/QueryIterHashJoin.java   | 148 ++++++++++++
 .../engine/join/QueryIterNestedLoopJoin.java    | 121 ++++++++++
 .../java/org/apache/jena/sparql/TC_General.java |   2 +
 .../jena/sparql/join/AbstractTestJoin.java      | 236 +++++++++++++++++++
 .../org/apache/jena/sparql/join/TS_Join.java    |  34 +++
 .../apache/jena/sparql/join/TestHashJoin.java   |  38 +++
 .../jena/sparql/join/TestJoinNestedLoop.java    |  34 +++
 .../sparql/join/TestJoinNestedLoopSimple.java   |  34 +++
 .../apache/jena/sparql/join/TestJoinSimple.java |  34 +++
 13 files changed, 1039 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/HashProbeTable.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/HashProbeTable.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/HashProbeTable.java
new file mode 100644
index 0000000..3238113
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/HashProbeTable.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.jena.atlas.iterator.Iter;
+import org.apache.jena.ext.com.google.common.collect.ArrayListMultimap;
+import org.apache.jena.ext.com.google.common.collect.Multimap;
+import org.apache.jena.sparql.engine.binding.Binding;
+
+/** The probe table for a hash join */
+class HashProbeTable {
+    private long s_count           = 0;
+    private long s_bucketCount     = 0;
+    private long s_maxBucketSize   = 0;
+    private long s_noKeyBucketSize = 0;
+    private long s_maxMatchGroup   = 0;
+
+    private final List<Binding>             noKeyBucket = new ArrayList<>();
+    private final Multimap<Object, Binding> buckets;
+    private final JoinKey                   joinKey;
+
+    HashProbeTable(JoinKey joinKey) {
+        this.joinKey = joinKey;
+        buckets = ArrayListMultimap.create();
+    }
+
+    public void put(Binding row) {
+        s_count++;
+        Object longHash = JoinLib.hash(joinKey, row);
+        if ( longHash == JoinLib.noKeyHash ) {
+            noKeyBucket.add(row);
+            return;
+        }
+        buckets.put(longHash, row);
+    }
+
+    public Iterator<Binding> getCandidates(Binding row) {
+        Iterator<Binding> iter = null;
+        Object longHash = JoinLib.hash(joinKey, row);
+        if ( longHash == JoinLib.noKeyHash )
+            iter = buckets.values().iterator();
+        else {
+            Collection<Binding> x = buckets.get(longHash);
+            if ( x != null ) {
+                s_maxMatchGroup = Math.max(s_maxMatchGroup, x.size());
+                iter = x.iterator();
+            }
+        }
+        // And the rows with no common hash key
+        if ( noKeyBucket != null )
+            iter = Iter.concat(iter, noKeyBucket.iterator());
+        return iter;
+    }
+
+    public void stats() {
+        long max = 0;
+        for ( Object key : buckets.keys() ) {
+            long s = buckets.get(key).size();
+            max = Math.max(max, s);
+        }
+        s_maxBucketSize = max;
+        s_bucketCount = buckets.keys().size();
+        s_noKeyBucketSize = (noKeyBucket == null) ? 0 : noKeyBucket.size();
+        // s_count
+        // s_maxMatchGroup
+        // What to do with them?
+    }
+
+    // Should not need these operations.
+    public Collection<Binding> getNoKey$() {
+        if ( noKeyBucket == null )
+            return null;
+        return noKeyBucket;
+    }
+
+    public Collection<Binding> getHashMatch$(Binding row) {
+        Object longHash = JoinLib.hash(joinKey, row);
+        if ( longHash == JoinLib.noKeyHash )
+            return noKeyBucket;
+        Collection<Binding> list = buckets.get(longHash);
+        return list;
+    }
+
+    public void clear() {
+        buckets.clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
new file mode 100644
index 0000000..0a71cc1
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import java.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.lib.DS ;
+import org.apache.jena.sparql.algebra.Algebra ;
+import org.apache.jena.sparql.engine.ExecutionContext ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.binding.Binding ;
+import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
+
+/** API to various join algorithms */
+public class Join {
+    // See also package org.apache.jena.sparql.engine.index
+    
+    /** Evaluate a hash join.  This code materializes the left into a probe table
+     * then hash joins from the right.  
+     * 
+     * @param joinKey   The key for the probe table.
+     * @param left      Left hand side
+     * @param right     Right hand side
+     * @param cxt       ExecutionContext
+     * @return          QueryIterator
+     */
+    public static QueryIterator hashJoin(JoinKey joinKey, QueryIterator left, QueryIterator right, ExecutionContext cxt) {
+        return new QueryIterHashJoin(joinKey, left, right, cxt) ;
+    }
+
+    /** Very simple, materializing version - useful for debugging. 
+     * Does <b>not</b> scale. 
+     */
+    public static QueryIterator innerLoopJoinBasic(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
+        List<Binding> leftRows = Iter.toList(left) ;
+        List<Binding> output = DS.list() ;
+        for ( ; right.hasNext() ; ) {
+            Binding row2 = right.next() ;
+            for ( Binding row1 : leftRows ) {
+                Binding r = Algebra.merge(row1, row2) ;
+                if ( r != null )
+                    output.add(r) ;
+            }
+        }
+        return new QueryIterPlainWrapper(output.iterator(), cxt) ;
+    }
+
+    /** Inner loops join.  This code materializes the left ino the inner loop tabole and
+     *  then streams on the right.
+     *  
+     * @param left      Left hand side
+     * @param right     Right hand side
+     * @param cxt       ExecutionContext
+     * @return          QueryIterator
+     */ 
+    public static QueryIterator innerLoopJoin(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
+        return new QueryIterNestedLoopJoin(left, right, cxt) ;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/JoinKey.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/JoinKey.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/JoinKey.java
new file mode 100644
index 0000000..587fc0a
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/JoinKey.java
@@ -0,0 +1,108 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+import java.util.Collection ;
+import java.util.Iterator ;
+import java.util.List ;
+
+import org.apache.jena.atlas.lib.DS ;
+
+import org.apache.jena.sparql.core.Var ;
+
+/** JoinKey for hash joins */
+public final class JoinKey implements Iterable<Var>
+{
+    // Common way to make a JoinKey
+    /** Make a JoinKey from the intersection of two sets **/  
+    
+    public static JoinKey create(Collection<Var> vars1, Collection<Var> vars2) {
+        // JoinKeys are generally small so short loops are best.
+        // vars2 may be smallest e.g. from triple and running accumulator (vars1) 
+        List<Var> intersection = DS.list() ;
+        for ( Var v : vars1 ) {
+            if ( vars2.contains(v) )
+                intersection.add(v) ;
+        }
+        return new JoinKey(intersection) ;
+    }
+    
+    public static JoinKey create(Var var) {
+        return new JoinKey(var) ;
+    }
+    
+    /** The builder can emit a key every time build() is caller
+     * and it can be continued to be used.
+     */
+    public static final class Builder {
+        private List<Var> keys = DS.list() ;
+        
+        public Builder() { }
+        
+        public boolean contains(Var var) {
+            return keys.contains(var) ;
+        }
+        
+        public Builder add(Var var) {
+            // We expect the keys list to be short - a Set is overkill(??)
+            if ( ! contains(var) )
+                keys.add(var) ;
+            return this ;
+        }
+        
+        public Builder remove(Var var) {
+            keys.remove(var) ;
+            return this ;
+        }
+
+        public Builder clear()      { keys.clear() ; return this ; }
+
+        public JoinKey build() {
+            JoinKey joinKey = new JoinKey(DS.list(keys)) ; 
+            return joinKey ;
+        }
+    }
+    
+    // Consider using an array.
+    private final List<Var> keys ;
+    
+    private JoinKey(List<Var> _keys) { keys = _keys ; }
+    
+    private JoinKey(Var var)     { keys = DS.listOfOne(var) ; }
+    
+    public boolean isEmpty()    { return keys.isEmpty() ; }
+
+    /** Get a single variable for this key. 
+     *  For any one key, it always returns the same var */ 
+    public Var getVarKey() { 
+        if ( keys.isEmpty() )
+            return null ;
+        return keys.get(0) ;
+    }
+    
+    @Override
+    public Iterator<Var> iterator() { return keys.iterator() ; }
+    
+    @Override
+    public String toString() {
+        return keys.toString() ;
+    }
+}
+
+
+

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/JoinLib.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/JoinLib.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/JoinLib.java
new file mode 100644
index 0000000..645e381
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/JoinLib.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.graph.Node ;
+import org.apache.jena.sparql.core.Var ;
+import org.apache.jena.sparql.engine.binding.Binding ;
+
+/** Internal operations in support of join algorithms. */
+class JoinLib {
+
+    /** Control stats output / development use */ 
+    static final boolean JOIN_EXPLAIN = false;
+
+    // No hash key marker.
+    public static final Object noKeyHash = new Object() ;
+    public static final long nullHashCode = 5 ;
+
+    public static long hash(Var v, Node x) {
+        long h = 17;
+        if ( v != null )
+            h = h ^ v.hashCode();
+        if ( x != null )
+            h = h ^ x.hashCode();
+        return h;
+    }
+
+    public static Object hash(JoinKey joinKey, Binding row) {
+          long x = 31 ;
+          boolean seenJoinKeyVar = false ; 
+          // Neutral to order in the set.
+          for ( Var v : joinKey ) {
+              Node value = row.get(v) ;
+              long h = nullHashCode ;
+              if ( value != null ) {
+                  seenJoinKeyVar = true ;
+                  h = hash(v, value) ;
+              } else {
+                  // In join key, not in row.
+              }
+                  
+              x = x ^ h ;
+          }
+          if ( ! seenJoinKeyVar )
+              return noKeyHash ;
+          return x ;
+      }
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java
new file mode 100644
index 0000000..cf8a520
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java
@@ -0,0 +1,148 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import java.util.Iterator ;
+
+import org.apache.jena.sparql.algebra.Algebra ;
+import org.apache.jena.sparql.engine.ExecutionContext ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.binding.Binding ;
+import org.apache.jena.sparql.engine.iterator.QueryIter2 ;
+
+/** Hash join.  This code materializes the left into a probe table
+ * then hash joins from the right.
+ */  
+public class QueryIterHashJoin extends QueryIter2 {
+    private long s_countLHS             = 0 ;       // Left input side size
+    private long s_countRHS             = 0 ;       // Right input side size
+    private long s_countResults         = 0 ;       // Result size.
+    private long s_bucketCount          = 0 ;
+    private long s_maxBucketSize        = 0 ;
+    private long s_noKeyBucketSize      = 0 ;
+    private long s_maxMatchGroup        = 0 ;
+    private long s_countRightMiss       = 0 ;
+    
+    private final JoinKey               joinKey ;
+    private final HashProbeTable        hashTable ;
+
+    private Iterator<Binding>           iterRight ;
+    private Binding                     rowRight          = null ;
+    private Iterator<Binding>           iterCurrent ;
+    
+    private Binding slot = null ;
+    private boolean finished = false ; 
+
+    public QueryIterHashJoin(JoinKey joinKey, QueryIterator left, QueryIterator right, ExecutionContext cxt) {
+        super(left, right, cxt) ;
+        this.joinKey = joinKey ;
+        this.iterRight = right ;
+        this.hashTable = new HashProbeTable(joinKey) ;
+        this.iterCurrent = null ;
+        phase1(left) ;
+    }
+        
+    private void phase1(Iterator<Binding> iter1) {
+        // Phase 1 : Build hash table. 
+        for (; iter1.hasNext();) {
+            Binding row1 = iter1.next() ;
+            s_countLHS ++ ;
+            hashTable.put(row1) ;
+        }
+    }
+
+    @Override
+    protected boolean hasNextBinding() {
+        if ( finished ) 
+            return false ;
+        if ( slot == null ) {
+            slot = moveToNextBindingOrNull() ;
+            if ( slot == null ) {
+                close() ;
+                return false;
+            }
+        }
+        return true ;
+    }
+
+    @Override
+    protected Binding moveToNextBinding() {
+        Binding r = slot ;
+        slot = null ;
+        return r ;
+    }
+
+    protected Binding moveToNextBindingOrNull() {
+        // Gather stats
+        // Internal IteratorSlotted.ended call?
+        // iterCurrent is the iterator of entries in the left hashed table
+        // for the right row.    
+        // iterRight is the stream of incoming rows.
+        for(;;) {
+            // Ensure we are processing a row. 
+            while ( iterCurrent == null ) {
+                // Move on to the next row from the right.
+                if ( ! iterRight.hasNext() ) {
+                    joinFinished() ;
+                    return null ;
+                }
+                
+                rowRight = iterRight.next() ;    
+                s_countRHS ++ ;
+                iterCurrent = hashTable.getCandidates(rowRight) ;
+            }
+            
+            // Emit one row using the rightRow and the current matched left rows. 
+            if ( ! iterCurrent.hasNext() ) {
+                iterCurrent = null ;
+                continue ;
+            }
+
+            Binding rowLeft = iterCurrent.next() ;
+            Binding r = Algebra.merge(rowLeft, rowRight) ;
+            if (r != null) {
+                s_countResults ++ ;
+                return r ;
+            } else {
+                // XXX LeftJoin
+                // return rowLeft ;
+            }
+        }
+    }        
+
+    private void joinFinished() {
+    }
+        
+    @Override
+    protected void closeSubIterator() {
+        finished = true ;
+        if ( JoinLib.JOIN_EXPLAIN ) {
+            String x = String.format(
+                         "HashJoin: LHS=%d RHS=%d Results=%d RightMisses=%d MaxBucket=%d NoKeyBucket=%d",
+                         s_countLHS, s_countRHS, s_countResults, 
+                         s_countRightMiss, s_maxBucketSize, s_noKeyBucketSize) ;
+            System.out.println(x) ;
+        }
+    }
+
+    @Override
+    protected void requestSubCancel() { 
+        finished = true ;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java
new file mode 100644
index 0000000..0208697
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.jena.atlas.iterator.Iter;
+import org.apache.jena.sparql.algebra.Algebra;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.QueryIterator;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.iterator.QueryIter2;
+
+/**
+ * Nested Loop Join (materializing on the left, streaming on the right)
+ * A simple, dependable join.
+ * <p>
+ * See {@link Join#innerLoopJoinBasic} for a very simple implementation for 
+ * testing purposes only. 
+ */
+public class QueryIterNestedLoopJoin extends QueryIter2 {
+    private long s_countLHS     = 0;
+    private long s_countRHS     = 0;
+    private long s_countResults = 0;
+
+    private final List<Binding> leftRows;
+    private Iterator<Binding>   left     = null;
+    private QueryIterator       right;
+    private Binding             rowRight = null;
+
+    private Binding slot     = null;
+    private boolean finished = false;
+
+    public QueryIterNestedLoopJoin(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
+        super(left, right, cxt);
+        List<Binding> rowsLeftList = Iter.toList(left);
+        leftRows = rowsLeftList;
+        s_countLHS = rowsLeftList.size();
+        this.right = right;
+    }
+
+    @Override
+    protected boolean hasNextBinding() {
+        if ( finished )
+            return false;
+        if ( slot == null ) {
+            slot = moveToNextBindingOrNull();
+            if ( slot == null ) {
+                close();
+                return false;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    protected Binding moveToNextBinding() {
+        Binding r = slot;
+        slot = null;
+        return r;
+    }
+
+    protected Binding moveToNextBindingOrNull() {
+        if ( isFinished() )
+            return null;
+
+        for ( ;; ) { // For rows from the right.
+            if ( rowRight == null ) {
+                if ( right.hasNext() ) {
+                    rowRight = right.next();
+                    s_countRHS++;
+                    left = leftRows.iterator();
+                } else
+                    return null;
+            }
+
+            // There is a rowRight
+            while (left.hasNext()) {
+                Binding rowLeft = left.next();
+                Binding r = Algebra.merge(rowLeft, rowRight);
+                if ( r != null ) {
+                    s_countResults++;
+                    return r;
+                } else {
+                    // XXX LeftJoin
+                    //return rowLeft ;
+                }
+            }
+            // Nothing more for this rowRight.
+            rowRight = null;
+        }
+    }
+
+    @Override
+    protected void requestSubCancel() {}
+
+    @Override
+    protected void closeSubIterator() {
+        if ( JoinLib.JOIN_EXPLAIN ) {
+            String x = String.format("InnerLoopJoin: LHS=%d RHS=%d Results=%d", s_countLHS, s_countRHS, s_countResults);
+            System.out.println(x);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java b/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java
index 286801b..7c67a5d 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java
@@ -30,6 +30,7 @@ import org.apache.jena.sparql.expr.NodeValue ;
 import org.apache.jena.sparql.expr.TS_Expr ;
 import org.apache.jena.sparql.function.user.TS_UserFunctions ;
 import org.apache.jena.sparql.graph.TS_Graph ;
+import org.apache.jena.sparql.join.TS_Join ;
 import org.apache.jena.sparql.lang.TS_Lang ;
 import org.apache.jena.sparql.modify.TS_Update ;
 import org.apache.jena.sparql.negation.TS_Negation ;
@@ -61,6 +62,7 @@ import org.junit.runners.Suite ;
     , TS_Negation.class
     , TS_Solver.class
     , TS_Algebra.class
+    , TS_Join.class
     , TS_Optimization.class
     , TS_ResultSet.class
     , TS_Syntax.class

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/test/java/org/apache/jena/sparql/join/AbstractTestJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/AbstractTestJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/AbstractTestJoin.java
new file mode 100644
index 0000000..a15d3e3
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/join/AbstractTestJoin.java
@@ -0,0 +1,236 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.join;
+
+import java.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.query.ResultSet ;
+import org.apache.jena.query.ResultSetFactory ;
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.algebra.TableFactory ;
+import org.apache.jena.sparql.core.Var ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.binding.Binding ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.resultset.ResultSetCompare ;
+import org.apache.jena.sparql.sse.SSE ;
+import org.junit.Assert ;
+import org.junit.Test ;
+
+public abstract class AbstractTestJoin extends Assert {
+    static Var var_a = Var.alloc("a") ; 
+    static Var var_b = Var.alloc("b") ; 
+    static Var var_c = Var.alloc("c") ; 
+    static Var var_d = Var.alloc("d") ; 
+
+    static Table table0() { return parseTableInt("(table)") ; } 
+
+    // For Mere, these must be in sort-joinkey order.
+
+    private static Table parseTableInt(String... strings) {
+        String x = StrUtils.strjoinNL(strings) ;
+        return SSE.parseTable(x) ;
+    }
+
+    // Table of one row and no colums.
+    static Table table1() { 
+        return parseTableInt("(table (row))") ; }
+
+    static Table tableD1() { 
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?b 2))",
+                             "   (row (?a 1) (?b 3))",
+                             "   (row (?a 1) (?b 2))",
+            ")") ;
+    }
+
+    static Table tableD2() { 
+        return parseTableInt("(table", 
+                             "   (row (?a 0) (?d 8))",
+                             "   (row (?a 1) (?c 9))",
+            ")") ;
+    }
+
+    static Table tableD3() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 9) (?b 2))",
+                             "   (row (?a 1) (?c 9) (?b 3))",
+                             "   (row (?a 1) (?c 9) (?b 2))",
+            ")") ;
+    }
+
+    static Table tableD4() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?b 2))",
+                             "   (row (?a 1) (?b 3))",
+                             "   (row (?a 4) (?b 4))",
+                             "   (row (?a 4) (?b 5))",
+            ")") ;
+    }
+
+    static Table tableD5() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4))",
+                             "   (row (?a 4) (?c 5))",
+                             "   (row (?a 6) (?c 5))",
+            ")") ;
+    }
+
+    static Table tableD6() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 2))",
+                             "   (row (?a 1) (?c 3))",
+                             "   (row (?a 4) (?c 4))",
+                             "   (row (?a 4) (?c 5))",
+            ")") ;
+    }
+
+    static Table tableD4x5() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+            ")") ;
+    }
+
+    static Table tableD4x6() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 2) (?b 2))",
+                             "   (row (?a 1) (?c 2) (?b 3))",
+                             "   (row (?a 1) (?c 3) (?b 2))",
+                             "   (row (?a 1) (?c 3) (?b 3))",
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+            ")") ;
+    }
+
+    // Disjoint.
+    static Table tableD8() {
+        return parseTableInt("(table",
+                             "  (row (?x 10))",
+                             "  (row (?z 11))",
+            ")") ; 
+    }
+
+    // Table8 crossproduct table2
+    static Table tableD8x2() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?d 8) (?z 11))",
+                             "  (row (?a 0) (?d 8) (?x 10))",
+                             "  (row (?a 1) (?c 9) (?z 11))",
+                             "  (row (?a 1) (?c 9) (?x 10))",
+            ")") ;
+    }
+
+    @Test public void join_00()  { testJoin("a", table0(), table0(), table0()) ; }
+    @Test public void join_00a() { testJoin("a", table1(), table0(), table0()) ; }
+    @Test public void join_00b() { testJoin("a", tableD1(), table1(), tableD1()) ; }
+    @Test public void join_00c() { testJoin("z", tableD1(), table1(), tableD1()) ; }
+
+    @Test public void join_01() { testJoin("a", table0(), tableD2(), table0()) ; }
+    @Test public void join_02() { testJoin("a", tableD1(), table0(), table0()) ; }
+    @Test public void join_03() { testJoin("a", tableD1(), tableD2(), tableD3()) ; }
+
+    // Identity.
+    @Test public void join_04() { testJoin("a", tableD2(), table1(), tableD2()) ; }
+    @Test public void join_05() { testJoin("a", table1(), tableD2(), tableD2()) ; }
+    // Identity, keymiss
+    @Test public void join_06() { testJoin("z", table1(), tableD2(), tableD2()) ; }
+    @Test public void join_07() { testJoin("z", tableD2(), table1(), tableD2()) ; }
+
+    @Test public void join_08() { testJoin("a", tableD4(), tableD5(), tableD4x5()) ; }
+    @Test public void join_09() { testJoin("a", tableD5(), tableD4(), tableD4x5()) ; }
+
+    @Test public void join_10() { testJoin("a", tableD4(), tableD6(), tableD4x6()) ; }
+    @Test public void join_11() { testJoin("a", tableD6(), tableD4(), tableD4x6()) ; }
+
+    // Not the right join key - should still work albeit less efficiently.
+    @Test public void join_12() { testJoin("z", tableD1(), tableD2(), tableD3()) ; }
+    @Test public void join_13() { testJoin("z", tableD2(), tableD1(), tableD3()) ; }
+
+    // No key.
+    @Test public void join_14() { testJoin(null, tableD1(), tableD2(), tableD3()) ; }
+
+
+    // Disjoint tables.
+    @Test public void join_disjoint_01() { testJoin("a", tableD2(), tableD8(), tableD8x2()) ; }
+    @Test public void join_disjoint_02() { testJoin("z", tableD2(), tableD8(), tableD8x2()) ; }
+
+    private void testJoin(String var, Table left, Table right, Table tableOut) {
+        JoinKey joinKey ;
+        if ( var != null ) {
+            if ( var.startsWith("?") )
+                var = var.substring(1) ;
+            joinKey = JoinKey.create(Var.alloc(var)) ;
+        }
+        else {
+            // No vars in join key.  Legal, albeit silly.
+            joinKey = new JoinKey.Builder().build() ;
+        }
+
+        testJoin1("1", joinKey, left, right, tableOut) ;
+        // Commumute
+        testJoin1("2", joinKey, right, left, tableOut) ;
+    }
+
+    private List<Binding> toList(Table table) {
+        return Iter.toList(table.rows()) ;
+    }
+
+    private void testJoin1(String num, JoinKey joinKey, Table left, Table right, Table expectedResults) {
+        Table x1 = joinMaterialize(joinKey, left, right) ;
+        assertNotNull("Null table from join ("+num+")", x1) ;
+        check("Results not equal ("+num+")", joinKey, left, right, expectedResults, x1) ;
+    }
+
+    private Table joinMaterialize(JoinKey joinKey, Table left, Table right) {
+        QueryIterator qIter = join(joinKey , left , right) ;
+        return TableFactory.create(qIter) ;
+    }
+
+    public abstract QueryIterator join(JoinKey joinKey , Table left , Table right) ;
+
+    private static void check(String msg, JoinKey joinKey, Table left, Table right, Table expected, Table actual) {
+        boolean b = equalTables(expected, actual) ;
+        if ( ! b ) {
+            System.out.println("Joinkey:  "+joinKey) ;
+            System.out.println("Left:     "+left) ;
+            System.out.println("Right:    "+right) ;
+            System.out.println("Expected: "+expected) ;
+            System.out.println("Actual:   "+actual) ;
+            System.out.println() ;
+        }
+
+        assertTrue(msg, b) ;
+    }
+
+    private static boolean equalTables(Table table1, Table table2) {
+        ResultSet rs1 =  ResultSetFactory.create(table1.iterator(null), table1.getVarNames()) ;
+        ResultSet rs2 =  ResultSetFactory.create(table2.iterator(null), table2.getVarNames()) ;
+        return ResultSetCompare.equalsByTerm(rs1, rs2) ;
+    }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/test/java/org/apache/jena/sparql/join/TS_Join.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TS_Join.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TS_Join.java
new file mode 100644
index 0000000..0a4c2a6
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/join/TS_Join.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.join;
+
+import org.junit.runner.RunWith ;
+import org.junit.runners.Suite ;
+import org.junit.runners.Suite.SuiteClasses ;
+
+@RunWith(Suite.class)
+@SuiteClasses( {
+    TestJoinSimple.class
+    , TestJoinNestedLoopSimple.class    // Real simple materializing version.
+    , TestJoinNestedLoop.class
+    , TestHashJoin.class
+})
+
+public class TS_Join { }
+

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/test/java/org/apache/jena/sparql/join/TestHashJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestHashJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestHashJoin.java
new file mode 100644
index 0000000..8358bc3
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestHashJoin.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.join;
+
+import java.util.Iterator ;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.binding.Binding ;
+import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
+import org.apache.jena.sparql.engine.join.Join ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+
+public class TestHashJoin extends AbstractTestJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        Iterator<Binding> data = Join.hashJoin(joinKey, left.iterator(null), right.iterator(null), null) ;
+        QueryIterator qIter = new QueryIterPlainWrapper(data) ;
+        return qIter ;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoop.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoop.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoop.java
new file mode 100644
index 0000000..4123c1c
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoop.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.join.Join ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+
+public class TestJoinNestedLoop extends AbstractTestJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        return Join.innerLoopJoin(left.iterator(null), right.iterator(null), null) ;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoopSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoopSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoopSimple.java
new file mode 100644
index 0000000..ff56d13
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoopSimple.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.join.Join ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+
+public class TestJoinNestedLoopSimple extends AbstractTestJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        return Join.innerLoopJoinBasic(left.iterator(null), right.iterator(null), null) ;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/9f158b67/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinSimple.java
new file mode 100644
index 0000000..b7135cf
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinSimple.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.TableJoin ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+
+public class TestJoinSimple extends AbstractTestJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        return TableJoin.join(left.iterator(null), right, null, null) ;
+    }
+
+}
+


[08/20] jena git commit: Minor

Posted by an...@apache.org.
Minor


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/f25095c4
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/f25095c4
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/f25095c4

Branch: refs/heads/master
Commit: f25095c47d8d5209a9854631b38d7f2acfcb8c55
Parents: 24e4c1e
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 14:18:21 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:12 2015 +0100

----------------------------------------------------------------------
 .../jena/sparql/engine/join/QueryIterNestedLoopJoin.java  | 10 +++-------
 1 file changed, 3 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/f25095c4/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java
index 0208697..19684c3 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopJoin.java
@@ -32,7 +32,7 @@ import org.apache.jena.sparql.engine.iterator.QueryIter2;
  * Nested Loop Join (materializing on the left, streaming on the right)
  * A simple, dependable join.
  * <p>
- * See {@link Join#innerLoopJoinBasic} for a very simple implementation for 
+ * See {@link Join#nestedLoopLeftJoinBasic} for a very simple implementation for 
  * testing purposes only. 
  */
 public class QueryIterNestedLoopJoin extends QueryIter2 {
@@ -50,9 +50,8 @@ public class QueryIterNestedLoopJoin extends QueryIter2 {
 
     public QueryIterNestedLoopJoin(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
         super(left, right, cxt);
-        List<Binding> rowsLeftList = Iter.toList(left);
-        leftRows = rowsLeftList;
-        s_countLHS = rowsLeftList.size();
+        leftRows = Iter.toList(left);
+        s_countLHS = leftRows.size();
         this.right = right;
     }
 
@@ -98,9 +97,6 @@ public class QueryIterNestedLoopJoin extends QueryIter2 {
                 if ( r != null ) {
                     s_countResults++;
                     return r;
-                } else {
-                    // XXX LeftJoin
-                    //return rowLeft ;
                 }
             }
             // Nothing more for this rowRight.


[10/20] jena git commit: JENA-1023: Nest loop leftJoin to fit with cancellation etc.

Posted by an...@apache.org.
JENA-1023: Nest loop leftJoin to fit with cancellation etc.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/cc6cc0b0
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/cc6cc0b0
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/cc6cc0b0

Branch: refs/heads/master
Commit: cc6cc0b057301caed252a2aeb78388b60979816c
Parents: f25095c
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 14:18:44 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:16 2015 +0100

----------------------------------------------------------------------
 .../join/QueryIterNestedLoopLeftJoin.java       | 135 +++++++++++++++++++
 1 file changed, 135 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/cc6cc0b0/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopLeftJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopLeftJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopLeftJoin.java
new file mode 100644
index 0000000..338f6e9
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterNestedLoopLeftJoin.java
@@ -0,0 +1,135 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.jena.atlas.iterator.Iter;
+import org.apache.jena.sparql.algebra.Algebra;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.QueryIterator;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.iterator.QueryIter2;
+import org.apache.jena.sparql.expr.ExprList ;
+
+/**
+ * Nested Loop left Join (materializing on the right, streaming on the left)
+ * A simple, dependable join.
+ * <p>
+ * See {@link Join#nestedLoopLeftJoinBasic} for a very simple implementation for 
+ * testing purposes only. 
+ */
+public class QueryIterNestedLoopLeftJoin extends QueryIter2 {
+    // XXX Can we materialise left instead?
+    
+    private long s_countLHS     = 0;
+    private long s_countRHS     = 0;
+    private long s_countResults = 0;
+
+    private final ExprList conditions;
+    private final List<Binding> rightRows;
+    private Iterator<Binding>   right     = null;
+    private QueryIterator       left;
+    private Binding             rowLeft = null;
+    private boolean foundMatch ;
+
+    private Binding slot     = null;
+    private boolean finished = false;
+
+    public QueryIterNestedLoopLeftJoin(QueryIterator left, QueryIterator right, ExprList exprList, ExecutionContext cxt) {
+        super(left, right, cxt);
+        conditions = exprList ;
+        rightRows =  Iter.toList(right);
+        s_countRHS = rightRows.size();
+        this.left = left;
+    }
+
+    @Override
+    protected boolean hasNextBinding() {
+        if ( finished )
+            return false;
+        if ( slot == null ) {
+            slot = moveToNextBindingOrNull();
+            if ( slot == null ) {
+                close();
+                return false;
+            }
+        }
+        return true;
+    }
+
+    @Override
+    protected Binding moveToNextBinding() {
+        Binding r = slot;
+        slot = null;
+        return r;
+    }
+
+    protected Binding moveToNextBindingOrNull() {
+        if ( isFinished() )
+            return null;
+
+        for ( ;; ) { // For rows from the left
+            if ( rowLeft == null ) {
+                if ( left.hasNext() ) {
+                    rowLeft = left.next();
+                    foundMatch = false ;
+                    s_countLHS++;
+                    right = rightRows.iterator();
+                } else
+                    return null;
+            }
+
+            while (right.hasNext()) {
+                Binding rowRight = right.next();
+                Binding r = Algebra.merge(rowLeft, rowRight);
+                if ( r != null && applyConditions(r) ) {
+                    s_countResults++;
+                    foundMatch = true ;
+                    return r;
+                }
+            }
+            if ( ! foundMatch ) {
+                s_countResults++;
+                Binding r = rowLeft ;
+                rowLeft = null;    
+                return r ;
+            }
+            rowLeft = null;
+        }
+    }
+    
+    private boolean applyConditions(Binding binding) {
+        if ( conditions == null )
+            return true ;
+        return conditions.isSatisfied(binding, getExecContext()) ;
+    }
+    
+    @Override
+    protected void requestSubCancel() {}
+
+    @Override
+    protected void closeSubIterator() {
+        if ( JoinLib.JOIN_EXPLAIN ) {
+            String x = String.format("InnerLoopJoin: LHS=%d RHS=%d Results=%d", s_countLHS, s_countRHS, s_countResults);
+            System.out.println(x);
+        }
+    }
+}


[13/20] jena git commit: JENA-1023: Left join tests

Posted by an...@apache.org.
JENA-1023: Left join tests


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/56ea4f90
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/56ea4f90
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/56ea4f90

Branch: refs/heads/master
Commit: 56ea4f905391fa795c5cfdb651f83d91623337fa
Parents: 6da3c8d
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 14:19:35 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:56 2015 +0100

----------------------------------------------------------------------
 .../engine/join/AbstractTestInnerJoin.java      | 124 ++--------
 .../sparql/engine/join/AbstractTestJoin.java    | 231 +++++++++++++++++--
 .../engine/join/AbstractTestLeftJoin.java       | 153 +++---------
 .../apache/jena/sparql/engine/join/TS_Join.java |   5 +
 .../jena/sparql/engine/join/TestHashJoin.java   |   3 +-
 .../sparql/engine/join/TestJoinNestedLoop.java  |   5 +-
 .../engine/join/TestJoinNestedLoopSimple.java   |   5 +-
 .../jena/sparql/engine/join/TestJoinSimple.java |   3 +-
 .../engine/join/TestLeftJoinNestedLoop.java     |  35 +++
 .../join/TestLeftJoinNestedLoopSimple.java      |  35 +++
 .../sparql/engine/join/TestLeftJoinSimple.java  |   6 +-
 11 files changed, 348 insertions(+), 257 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java
index f5a1bdc..0242d4a 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java
@@ -19,123 +19,29 @@
 package org.apache.jena.sparql.engine.join;
 
 import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.core.Var ;
-import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.expr.ExprList ;
 import org.junit.Test ;
 
 /** Tests for inner/equi joins */ 
 public abstract class AbstractTestInnerJoin extends AbstractTestJoin {
     
     @Override
-    protected void executeTest(JoinKey joinKey, Table left, Table right, Table expectedResults) {
-        executeTestJoin("1", joinKey, left, right, expectedResults) ;
-        executeTestJoin("2", joinKey, right, left, expectedResults) ;
+    protected void executeTest(JoinKey joinKey, Table left, Table right, ExprList conditions, Table expectedResults) {
+        if ( conditions != null )
+            fail("Conditions on inner join are meaningless (currently)") ;
+        // No conditions.
+        // Commutative.
+        executeTestJoin("1", joinKey, left, right, null, expectedResults) ;
+        executeTestJoin("2", joinKey, right, left, null, expectedResults) ;
     }
     
-    static Var var_a = Var.alloc("a") ; 
-    static Var var_b = Var.alloc("b") ; 
-    static Var var_c = Var.alloc("c") ; 
-    static Var var_d = Var.alloc("d") ; 
-
-    static Table table0() { return parseTableInt("(table)") ; } 
-
-    // Table of one row and no colums.
-    static Table table1() { 
-        return parseTableInt("(table (row))") ; }
-
-    static Table tableD1() { 
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?b 2))",
-                             "   (row (?a 1) (?b 3))",
-                             "   (row (?a 1) (?b 2))",
-            ")") ;
-    }
-
-    static Table tableD2() { 
-        return parseTableInt("(table", 
-                             "   (row (?a 0) (?d 8))",
-                             "   (row (?a 1) (?c 9))",
-            ")") ;
-    }
-
-    static Table tableD3() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 9) (?b 2))",
-                             "   (row (?a 1) (?c 9) (?b 3))",
-                             "   (row (?a 1) (?c 9) (?b 2))",
-            ")") ;
-    }
-
-    static Table tableD4() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?b 2))",
-                             "   (row (?a 1) (?b 3))",
-                             "   (row (?a 4) (?b 4))",
-                             "   (row (?a 4) (?b 5))",
-            ")") ;
-    }
-
-    static Table tableD5() {
-        return parseTableInt("(table", 
-                             "   (row (?a 4) (?c 4))",
-                             "   (row (?a 4) (?c 5))",
-                             "   (row (?a 6) (?c 5))",
-            ")") ;
-    }
-
-    static Table tableD6() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 2))",
-                             "   (row (?a 1) (?c 3))",
-                             "   (row (?a 4) (?c 4))",
-                             "   (row (?a 4) (?c 5))",
-            ")") ;
-    }
-
-    static Table tableD4x5() {
-        return parseTableInt("(table", 
-                             "   (row (?a 4) (?c 4) (?b 4))",
-                             "   (row (?a 4) (?c 4) (?b 5))",
-                             "   (row (?a 4) (?c 5) (?b 4))",
-                             "   (row (?a 4) (?c 5) (?b 5))",
-            ")") ;
-    }
-
-    static Table tableD4x6() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 2) (?b 2))",
-                             "   (row (?a 1) (?c 2) (?b 3))",
-                             "   (row (?a 1) (?c 3) (?b 2))",
-                             "   (row (?a 1) (?c 3) (?b 3))",
-                             "   (row (?a 4) (?c 4) (?b 4))",
-                             "   (row (?a 4) (?c 4) (?b 5))",
-                             "   (row (?a 4) (?c 5) (?b 4))",
-                             "   (row (?a 4) (?c 5) (?b 5))",
-            ")") ;
-    }
-
-    // Disjoint.
-    static Table tableD8() {
-        return parseTableInt("(table",
-                             "  (row (?x 10))",
-                             "  (row (?z 11))",
-            ")") ; 
-    }
-
-    // Table8 crossproduct table2
-    static Table tableD8x2() {
-        return parseTableInt("(table",
-                             "  (row (?a 0) (?d 8) (?z 11))",
-                             "  (row (?a 0) (?d 8) (?x 10))",
-                             "  (row (?a 1) (?c 9) (?z 11))",
-                             "  (row (?a 1) (?c 9) (?x 10))",
-            ")") ;
-    }
-
-    @Test public void join_00()  { testJoin("a", table0(), table0(), table0()) ; }
-    @Test public void join_00a() { testJoin("a", table1(), table0(), table0()) ; }
-    @Test public void join_00b() { testJoin("a", tableD1(), table1(), tableD1()) ; }
-    @Test public void join_00c() { testJoin("z", tableD1(), table1(), tableD1()) ; }
+    @Test public void join_basic_1()    { testJoin("a", table0(), table0(), table0()) ; }
+    @Test public void join_basic_2()    { testJoin("a", table1(), table0(), table0()) ; }
+    @Test public void join_basic_3()    { testJoin("a", tableD1(), table1(), tableD1()) ; }
+    @Test public void join_basic_4()    { testJoin("z", tableD1(), table1(), tableD1()) ; }
+    
+    @Test public void join_basic_5() { testJoin("a", table0(), table1(), table0()) ; }
+    @Test public void join_basic_6() { testJoin("a", table1(), table0(), table0()) ; }
 
     @Test public void join_01() { testJoin("a", table0(), tableD2(), table0()) ; }
     @Test public void join_02() { testJoin("a", tableD1(), table0(), table0()) ; }

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
index b48c66b..2ed9647 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
@@ -29,19 +29,205 @@ import org.apache.jena.sparql.algebra.TableFactory ;
 import org.apache.jena.sparql.core.Var ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.binding.Binding ;
-import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.expr.ExprList ;
 import org.apache.jena.sparql.resultset.ResultSetCompare ;
 import org.apache.jena.sparql.sse.SSE ;
 import org.junit.Assert ;
 
 /** Tests for inner/equi joins */ 
 public abstract class AbstractTestJoin extends Assert {
+
+    protected static Table table0() { return parseTableInt("(table)") ; } 
+
+    // Table of one row and no colums.
+    protected static Table table1() { 
+        return parseTableInt("(table (row))") ; }
+
+    protected static Table tableD1() { 
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?b 2))",
+                             "   (row (?a 1) (?b 3))",
+                             "   (row (?a 1) (?b 2))",
+            ")") ;
+    }
+
+    protected static Table tableD2() { 
+        return parseTableInt("(table", 
+                             "   (row (?a 0) (?d 8))",
+                             "   (row (?a 1) (?c 9))",
+            ")") ;
+    }
+
+    protected static Table tableD3() {
+        return parseTableInt("(table",
+                             "   (row (?a 1) (?c 9) (?b 2))",
+                             "   (row (?a 1) (?c 9) (?b 3))",
+                             "   (row (?a 1) (?c 9) (?b 2))",
+            ")") ;
+    }
+
+    protected static Table tableD3_LJ() {
+        return parseTableInt("(table", 
+                             "   (row (?d 8) (?a 0))",
+                             "   (row (?a 1) (?c 9) (?b 2))",
+                             "   (row (?a 1) (?c 9) (?b 3))",
+                             "   (row (?a 1) (?c 9) (?b 2))",
+                             
+            ")") ;
+    }
+
+    protected static Table tableD4() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?b 2))",
+                             "   (row (?a 1) (?b 3))",
+                             "   (row (?a 4) (?b 4))",
+                             "   (row (?a 4) (?b 5))",
+            ")") ;
+    }
+
+    protected static Table tableD5() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4))",
+                             "   (row (?a 4) (?c 5))",
+                             "   (row (?a 6) (?c 5))",
+            ")") ;
+    }
+
+    protected static Table tableD6() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 2))",
+                             "   (row (?a 1) (?c 3))",
+                             "   (row (?a 4) (?c 4))",
+                             "   (row (?a 4) (?c 5))",
+            ")") ;
+    }
+
+    protected static Table tableD4x5() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+            ")") ;
+    }
+
+    protected static Table tableD4x5_LJ() {
+        return parseTableInt("(table",
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+                             "   (row (?b 2) (?a 1))",
+                             "   (row (?b 3) (?a 1))",
+            ")") ;
+    }
+
+    protected static Table tableD5x4_LJ() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+                             "   (row (?a 6) (?c 5))",
+            ")") ;
+    }
+
+    protected static Table tableD4x6() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 2) (?b 2))",
+                             "   (row (?a 1) (?c 2) (?b 3))",
+                             "   (row (?a 1) (?c 3) (?b 2))",
+                             "   (row (?a 1) (?c 3) (?b 3))",
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+            ")") ;
+    }
+
+    // Disjoint.
+    protected static Table tableD8() {
+        return parseTableInt("(table",
+                             "  (row (?x 10))",
+                             "  (row (?z 11))",
+            ")") ; 
+    }
+
+    // Table8 crossproduct table2
+    protected static Table tableD8x2() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?d 8) (?z 11))",
+                             "  (row (?a 0) (?d 8) (?x 10))",
+                             "  (row (?a 1) (?c 9) (?z 11))",
+                             "  (row (?a 1) (?c 9) (?x 10))",
+            ")") ;
+    }
+    
+    // Left join data tables.
+    protected static Table tableL1() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?d 8))",
+                             "  (row (?a 3) (?d 9))",
+            ")") ;
+    }
+
+    protected static Table tableL2() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?z 11))",
+                             "  (row (?a 1) (?c 9) (?z 11))",
+            ")") ;
+    }
+
+    // L3 := L1 leftjoin L2 
+    protected static Table table1LJ2() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?d 8) (?z 11))",
+                             "  (row (?a 3) (?d 9))",
+            ")") ;
+    }
+    
+    protected static Table tableL4() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?z 11))",
+                             "  (row (?a 0) (?z 12))",
+                             "  (row               (?r 99))",
+                             "  (row        (?c 9) (?z 11))",
+            ")") ;
+    }
+
+    protected static Table tableL5() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?d 8))",
+                             "  (row (?a 1) (?c 9) (?z 11))",
+            ")") ;
+    }
+
+    // L3 := L1 leftjoin L2 
+    protected static Table table4LJ5() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?d 8) (?z 11))",
+                             "  (row (?a 0) (?d 8) (?z 12))",
+                             
+                             "  (row (?a 0) (?d 8) (?r 99))",
+                             "  (row (?a 1) (?c 9) (?z 11) (?r 99))",
+                             
+                             "  (row (?a 0) (?d 8) (?c 9) (?z 11))",
+                             "  (row (?a 1) (?c 9) (?z 11))",
+            ")") ;
+    }
+    
+    // Code
+
     protected static Table parseTableInt(String... strings) {
         String x = StrUtils.strjoinNL(strings) ;
         return SSE.parseTable(x) ;
     }
 
     protected void testJoin(String var, Table left, Table right, Table tableOut) {
+        testJoin(var, left, right, null, tableOut); 
+    }
+    
+    protected void testJoin(String var, Table left, Table right, ExprList conditions, Table tableOut) {
         JoinKey joinKey ;
         if ( var != null ) {
             if ( var.startsWith("?") )
@@ -53,38 +239,49 @@ public abstract class AbstractTestJoin extends Assert {
             joinKey = new JoinKey.Builder().build() ;
         }
 
-        executeTest(joinKey, left, right, tableOut) ;
+        executeTest(joinKey, left, right, null, tableOut) ;
     }
 
-    protected abstract void executeTest(JoinKey joinKey, Table left, Table right, Table expectedResults) ;
-    
+    // Any kind of join (choose by abstract join() operation).
+    protected abstract void executeTest(JoinKey joinKey, Table left, Table right, ExprList conditions, Table expectedResults) ;
     
     private List<Binding> toList(Table table) {
         return Iter.toList(table.rows()) ;
     }
 
-    protected void executeTestJoin(String num, JoinKey joinKey, Table left, Table right, Table expectedResults) {
-        Table x1 = joinMaterialize(joinKey, left, right) ;
-        assertNotNull("Null table from join ("+num+")", x1) ;
-        check("Results not equal ("+num+")", joinKey, left, right, expectedResults, x1) ;
+    protected void executeTestJoin(String msg, JoinKey joinKey, Table left, Table right, ExprList conditions, Table expectedResults) {
+        Table x1 = joinMaterialize(joinKey, left, right, conditions) ;
+        assertNotNull("Null table from join ("+msg+")", x1) ;
+        if ( false ) {
+            System.out.println("Test :    "+msg) ;
+            System.out.println("Joinkey:  "+joinKey) ;
+            System.out.println("Left:     \n"+left) ;
+            System.out.println("Right:    \n"+right) ;
+            System.out.println("Expected: \n"+expectedResults) ;
+            System.out.println("Actual:   \n"+x1) ;
+            System.out.println() ;
+        }
+        
+        check("Results not equal ("+msg+")", joinKey, left, right, expectedResults, x1) ;
     }
 
-    private Table joinMaterialize(JoinKey joinKey, Table left, Table right) {
-        QueryIterator qIter = join(joinKey, left , right) ;
+    private Table joinMaterialize(JoinKey joinKey, Table left, Table right, ExprList conditions) {
+        QueryIterator qIter = join(joinKey, left , right, null) ;
         return TableFactory.create(qIter) ;
     }
 
-    public abstract QueryIterator join(JoinKey joinKey , Table left , Table right) ;
+    public abstract QueryIterator join(JoinKey joinKey, Table left , Table right, ExprList conditions) ;
 
     private static void check(String msg, JoinKey joinKey, Table left, Table right, Table expected, Table actual) {
         boolean b = equalTables(expected, actual) ;
         if ( ! b ) {
-            System.out.println("Joinkey:  "+joinKey) ;
-            System.out.println("Left:     \n"+left) ;
-            System.out.println("Right:    \n"+right) ;
-            System.out.println("Expected: \n"+expected) ;
-            System.out.println("Actual:   \n"+actual) ;
-            System.out.println() ;
+            System.out.flush() ;
+            System.err.println("Joinkey:  "+joinKey) ;
+            System.err.println("Left:     \n"+left) ;
+            System.err.println("Right:    \n"+right) ;
+            System.err.println("Expected: \n"+expected) ;
+            System.err.println("Actual:   \n"+actual) ;
+            System.err.println() ;
         }
 
         assertTrue(msg, b) ;

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java
index 57756ff..2786bed 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java
@@ -19,146 +19,55 @@
 package org.apache.jena.sparql.engine.join;
 
 import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.core.Var ;
-import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.expr.ExprList ;
 import org.junit.Test ;
 
 public abstract class AbstractTestLeftJoin extends AbstractTestJoin {
-    @Override
-    protected void executeTest(JoinKey joinKey, Table left, Table right, Table expectedResults) {
-        executeTestJoin("LJ", joinKey, left, right, expectedResults) ;
-    }
     
-    static Var var_a = Var.alloc("a") ; 
-    static Var var_b = Var.alloc("b") ; 
-    static Var var_c = Var.alloc("c") ; 
-    static Var var_d = Var.alloc("d") ; 
-
-    static Table table0() { return parseTableInt("(table)") ; } 
-
-    // Table of one row and no colums.
-    static Table table1() { 
-        return parseTableInt("(table (row))") ; }
-
-    static Table tableD1() { 
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?b 2))",
-                             "   (row (?a 1) (?b 3))",
-                             "   (row (?a 1) (?b 2))",
-            ")") ;
-    }
-
-    static Table tableD2() { 
-        return parseTableInt("(table", 
-                             "   (row (?a 0) (?d 8))",
-                             "   (row (?a 1) (?c 9))",
-            ")") ;
-    }
-
-    static Table tableD3() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 9) (?b 2))",
-                             "   (row (?a 1) (?c 9) (?b 3))",
-                             "   (row (?a 1) (?c 9) (?b 2))",
-            ")") ;
-    }
-
-    static Table tableD4() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?b 2))",
-                             "   (row (?a 1) (?b 3))",
-                             "   (row (?a 4) (?b 4))",
-                             "   (row (?a 4) (?b 5))",
-            ")") ;
-    }
-
-    static Table tableD5() {
-        return parseTableInt("(table", 
-                             "   (row (?a 4) (?c 4))",
-                             "   (row (?a 4) (?c 5))",
-                             "   (row (?a 6) (?c 5))",
-            ")") ;
-    }
-
-    static Table tableD6() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 2))",
-                             "   (row (?a 1) (?c 3))",
-                             "   (row (?a 4) (?c 4))",
-                             "   (row (?a 4) (?c 5))",
-            ")") ;
-    }
-
-    static Table tableD4x5() {
-        return parseTableInt("(table", 
-                             "   (row (?a 4) (?c 4) (?b 4))",
-                             "   (row (?a 4) (?c 4) (?b 5))",
-                             "   (row (?a 4) (?c 5) (?b 4))",
-                             "   (row (?a 4) (?c 5) (?b 5))",
-            ")") ;
-    }
-
-    static Table tableD4x6() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 2) (?b 2))",
-                             "   (row (?a 1) (?c 2) (?b 3))",
-                             "   (row (?a 1) (?c 3) (?b 2))",
-                             "   (row (?a 1) (?c 3) (?b 3))",
-                             "   (row (?a 4) (?c 4) (?b 4))",
-                             "   (row (?a 4) (?c 4) (?b 5))",
-                             "   (row (?a 4) (?c 5) (?b 4))",
-                             "   (row (?a 4) (?c 5) (?b 5))",
-            ")") ;
-    }
-
-    // Disjoint.
-    static Table tableD8() {
-        return parseTableInt("(table",
-                             "  (row (?x 10))",
-                             "  (row (?z 11))",
-            ")") ; 
-    }
-
-    // Table8 crossproduct table2
-    static Table tableD8x2() {
-        return parseTableInt("(table",
-                             "  (row (?a 0) (?d 8) (?z 11))",
-                             "  (row (?a 0) (?d 8) (?x 10))",
-                             "  (row (?a 1) (?c 9) (?z 11))",
-                             "  (row (?a 1) (?c 9) (?x 10))",
-            ")") ;
+    @Override
+    protected void executeTest(JoinKey joinKey, Table left, Table right, ExprList conditions, Table expectedResults) {
+        executeTestJoin("LJ", joinKey, left, right, conditions, expectedResults) ;
     }
 
     // XXX And conditions.
     
-    @Test public void leftjoin_00()  { testJoin("a", table0(), table0(), table0()) ; }
-    @Test public void leftjoin_00a() { testJoin("a", table1(), table0(), table0()) ; }
-    @Test public void leftjoin_00b() { testJoin("a", tableD1(), table1(), tableD1()) ; }
-    @Test public void leftjoin_00c() { testJoin("z", tableD1(), table1(), tableD1()) ; }
+    @Test public void leftjoin_basic_1()    { testJoin("a", table0(), table0(), table0()) ; }
+    @Test public void leftjoin_basic_2()    { testJoin("a", table1(), table0(), table1()) ; }
+    @Test public void leftjoin_basic_3()    { testJoin("a", tableD1(), table1(), tableD1()) ; }
+    @Test public void leftjoin_basic_4()    { testJoin("z", tableD1(), table1(), tableD1()) ; }
+    
+    @Test public void leftjoin_01()         { testJoin("a", tableL1(), tableL2(), table1LJ2()) ; }
+    @Test public void leftjoin_02()         { testJoin("a", tableL4(), tableL5(), table4LJ5()) ; }
+    
+    @Test public void leftjoin_03()         { testJoin("z", tableL1(), tableL2(), table1LJ2()) ; }
+    @Test public void leftjoin_04()         { testJoin("z", tableL4(), tableL5(), table4LJ5()) ; }
+    
+    @Test public void leftjoin_J01()        { testJoin("a", tableD1(), table1(), tableD1()) ; }
+    @Test public void leftjoin_J01a()       { testJoin("z", tableD1(), table1(), tableD1()) ; }
 
-    @Test public void leftjoin_01() { testJoin("a", table0(), tableD2(), table0()) ; }
-    @Test public void leftjoin_02() { testJoin("a", tableD1(), table0(), tableD1()) ; }
-    @Test public void leftjoin_03() { testJoin("a", tableD1(), tableD2(), tableD3()) ; }
+    @Test public void leftjoin_J02()        { testJoin("a", tableD1(), table0(), tableD1()) ; }
+    @Test public void leftjoin_J03()        { testJoin("a", tableD1(), tableD2(), tableD3()) ; }
 
     // Identity.
-    @Test public void leftjoin_04() { testJoin("a", tableD2(), table1(), tableD2()) ; }
-    @Test public void leftjoin_05() { testJoin("a", table1(), tableD2(), tableD2()) ; }
+    @Test public void leftjoin_J04()        { testJoin("a", tableD2(), table1(), tableD2()) ; }
+    @Test public void leftjoin_J05()        { testJoin("a", table1(), tableD2(), tableD2()) ; }
     // Identity, keymiss
-    @Test public void leftjoin_06() { testJoin("z", table1(), tableD2(), tableD2()) ; }
-    @Test public void leftjoin_07() { testJoin("z", tableD2(), table1(), tableD2()) ; }
+    @Test public void leftjoin_J06()        { testJoin("z", table1(), tableD2(), tableD2()) ; }
+    @Test public void leftjoin_J07()        { testJoin("z", tableD2(), table1(), tableD2()) ; }
 
-    @Test public void leftjoin_08() { testJoin("a", tableD4(), tableD5(), tableD4x5()) ; }
-    @Test public void leftjoin_09() { testJoin("a", tableD5(), tableD4(), tableD4x5()) ; }
+    @Test public void leftjoin_J08()        { testJoin("a", tableD4(), tableD5(), tableD4x5_LJ()) ; }
+    @Test public void leftjoin_J09()        { testJoin("a", tableD5(), tableD4(), tableD5x4_LJ()) ; }
 
-    @Test public void leftjoin_10() { testJoin("a", tableD4(), tableD6(), tableD4x6()) ; }
-    @Test public void leftjoin_11() { testJoin("a", tableD6(), tableD4(), tableD4x6()) ; }
+    @Test public void leftjoin_J10()        { testJoin("a", tableD4(), tableD6(), tableD4x6()) ; }
+    @Test public void leftjoin_J11()        { testJoin("a", tableD6(), tableD4(), tableD4x6()) ; }
 
     // Not the right join key - should still work albeit less efficiently.
-    @Test public void leftjoin_12() { testJoin("z", tableD1(), tableD2(), tableD3()) ; }
-    @Test public void leftjoin_13() { testJoin("z", tableD2(), tableD1(), tableD3()) ; }
+    @Test public void leftjoin_J12()        { testJoin("z", tableD1(), tableD2(), tableD3()) ; }
+    @Test public void leftjoin_J13()        { testJoin("z", tableD2(), tableD1(), tableD3_LJ()) ; }
 
     // No key.
-    @Test public void leftjoin_14() { testJoin(null, tableD1(), tableD2(), tableD3()) ; }
+    
+    @Test public void leftjoin_14()         { testJoin(null, tableD1(), tableD2(), tableD3()) ; }
 
 
     // Disjoint tables.

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java
index 69b5fb4..bd00872 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java
@@ -28,6 +28,11 @@ import org.junit.runners.Suite.SuiteClasses ;
     , TestJoinNestedLoopSimple.class    // Real simple materializing version.
     , TestJoinNestedLoop.class
     , TestHashJoin.class
+    
+    , TestLeftJoinSimple.class
+    , TestLeftJoinNestedLoopSimple.class    // Real simple materializing version.
+    , TestLeftJoinNestedLoop.class
+    //, TestLeftHashJoin.class
 })
 
 public class TS_Join { }

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java
index 4a4bd6f..6aa9eaa 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java
@@ -26,11 +26,12 @@ import org.apache.jena.sparql.engine.binding.Binding ;
 import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
 import org.apache.jena.sparql.engine.join.Join ;
 import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.expr.ExprList ;
 
 public class TestHashJoin extends AbstractTestInnerJoin {
 
     @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+    public QueryIterator join(JoinKey joinKey, Table left, Table right, ExprList conditions) {
         Iterator<Binding> data = Join.hashJoin(joinKey, left.iterator(null), right.iterator(null), null) ;
         QueryIterator qIter = new QueryIterPlainWrapper(data) ;
         return qIter ;

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java
index c844ea2..31530a4 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java
@@ -22,12 +22,13 @@ import org.apache.jena.sparql.algebra.Table ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.join.Join ;
 import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.expr.ExprList ;
 
 public class TestJoinNestedLoop extends AbstractTestInnerJoin {
 
     @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
-        return Join.innerLoopJoin(left.iterator(null), right.iterator(null), null) ;
+    public QueryIterator join(JoinKey joinKey, Table left, Table right, ExprList conditions) {
+        return Join.nestedLoopJoin(left.iterator(null), right.iterator(null), null) ;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java
index 0d9db54..64206cb 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java
@@ -22,12 +22,13 @@ import org.apache.jena.sparql.algebra.Table ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.join.Join ;
 import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.expr.ExprList ;
 
 public class TestJoinNestedLoopSimple extends AbstractTestInnerJoin {
 
     @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
-        return Join.innerLoopJoinBasic(left.iterator(null), right.iterator(null), null) ;
+    public QueryIterator join(JoinKey joinKey, Table left, Table right, ExprList conditions) {
+        return Join.nestedLoopJoinBasic(left.iterator(null), right.iterator(null), null) ;
     }
 
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java
index d091168..f89fa87 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java
@@ -22,11 +22,12 @@ import org.apache.jena.sparql.algebra.Table ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.join.JoinKey ;
 import org.apache.jena.sparql.engine.ref.TableJoin ;
+import org.apache.jena.sparql.expr.ExprList ;
 
 public class TestJoinSimple extends AbstractTestInnerJoin {
 
     @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+    public QueryIterator join(JoinKey joinKey, Table left, Table right, ExprList conditions) {
         return TableJoin.join(left.iterator(null), right, null, null) ;
     }
 

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinNestedLoop.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinNestedLoop.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinNestedLoop.java
new file mode 100644
index 0000000..a5053a7
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinNestedLoop.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.join.Join ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.expr.ExprList ;
+
+public class TestLeftJoinNestedLoop extends AbstractTestLeftJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right, ExprList conditions) {
+        return Join.nestedLoopLeftJoin(left.iterator(null), right.iterator(null), conditions, null) ;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinNestedLoopSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinNestedLoopSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinNestedLoopSimple.java
new file mode 100644
index 0000000..44d2d7d
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinNestedLoopSimple.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.join.Join ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.expr.ExprList ;
+
+public class TestLeftJoinNestedLoopSimple extends AbstractTestLeftJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right, ExprList conditions) {
+        return Join.nestedLoopLeftJoinBasic(left.iterator(null), right.iterator(null), conditions,null) ;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/56ea4f90/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java
index 041dfe5..f9fc763 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java
@@ -22,13 +22,13 @@ import org.apache.jena.sparql.algebra.Table ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.join.JoinKey ;
 import org.apache.jena.sparql.engine.ref.TableJoin ;
+import org.apache.jena.sparql.expr.ExprList ;
 
 public class TestLeftJoinSimple extends AbstractTestLeftJoin {
 
     @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
-        return TableJoin.leftJoin(left.iterator(null), right, null, null) ;
+    public QueryIterator join(JoinKey joinKey, Table left, Table right, ExprList conditions) {
+        return TableJoin.leftJoin(left.iterator(null), right, conditions, null) ;
     }
-
 }
 


[19/20] jena git commit: Remove old code.

Posted by an...@apache.org.
Remove old code.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/a1012452
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/a1012452
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/a1012452

Branch: refs/heads/master
Commit: a10124525387b748f3d2155bed21b03d0bca5e8d
Parents: bb9574e
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 17:56:25 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:09:44 2015 +0100

----------------------------------------------------------------------
 .../engine/main/iterator/QueryIterJoin.java     | 32 ---------
 .../engine/main/iterator/QueryIterJoinBase.java | 76 --------------------
 .../engine/main/iterator/QueryIterLeftJoin.java | 32 ---------
 3 files changed, 140 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/a1012452/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java
deleted file mode 100644
index 2b45202..0000000
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.engine.main.iterator;
-
-import org.apache.jena.sparql.algebra.JoinType ;
-import org.apache.jena.sparql.engine.ExecutionContext ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-
-/** Join by materializing the RHS - this is not streamed */
-public class QueryIterJoin extends QueryIterJoinBase
-{
-    public QueryIterJoin(QueryIterator left, QueryIterator right, ExecutionContext qCxt)
-    {
-        super(left, right, JoinType.INNER, null, qCxt) ;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a1012452/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java
deleted file mode 100644
index 3fb00dc..0000000
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.engine.main.iterator;
-
-import org.apache.jena.sparql.algebra.JoinType;
-import org.apache.jena.sparql.engine.ExecutionContext;
-import org.apache.jena.sparql.engine.QueryIterator;
-import org.apache.jena.sparql.engine.binding.Binding;
-import org.apache.jena.sparql.engine.iterator.QueryIter;
-import org.apache.jena.sparql.engine.join.Join;
-import org.apache.jena.sparql.expr.ExprList;
-
-/**
- * Join or LeftJoin by calculating both sides, then doing the join It usually
- * better to use substitute algorithm (not this QueryIterator in other words) as
- * that is effectively indexing from one side into the other.
- */
-public class QueryIterJoinBase extends QueryIter {
-    // This should be converted to a hash or sort-merge join.
-    private final QueryIterator left;
-    private final QueryIterator right;
-    private final QueryIterator result;
-
-    protected QueryIterJoinBase(QueryIterator left, QueryIterator right, JoinType joinType, ExprList exprs, ExecutionContext execCxt) {
-        super(execCxt);
-        this.left = left;
-        this.right = right;
-        this.result = calc(left, right, joinType, exprs, execCxt);
-    }
-
-    private static QueryIterator calc(QueryIterator left, QueryIterator right,
-                                      JoinType joinType, ExprList exprs,
-                                      ExecutionContext execCxt) {
-        return Join.joinWorker(left, right, joinType, exprs, execCxt);
-    }
-
-    @Override
-    protected boolean hasNextBinding() {
-        return result.hasNext();
-    }
-
-    @Override
-    protected Binding moveToNextBinding() {
-        return result.nextBinding();
-    }
-
-    @Override
-    protected void closeIterator() {
-        left.close();
-        right.close();
-        result.close();
-    }
-
-    @Override
-    protected void requestCancel() {
-        left.cancel();
-        right.cancel();
-        result.cancel();
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/a1012452/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterLeftJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterLeftJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterLeftJoin.java
deleted file mode 100644
index 4a442f7..0000000
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterLeftJoin.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.engine.main.iterator;
-
-import org.apache.jena.sparql.algebra.JoinType ;
-import org.apache.jena.sparql.engine.ExecutionContext ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-import org.apache.jena.sparql.expr.ExprList ;
-
-/** Left join by materializing the RHS */
-public class QueryIterLeftJoin extends QueryIterJoinBase
-{
-    public QueryIterLeftJoin(QueryIterator left, QueryIterator right, ExprList exprs, ExecutionContext qCxt) {
-        super(left, right, JoinType.LEFT, exprs, qCxt) ;
-    }
-}


[11/20] jena git commit: Rename in JoinType.

Posted by an...@apache.org.
Rename in JoinType.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/eccf873f
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/eccf873f
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/eccf873f

Branch: refs/heads/master
Commit: eccf873fd8ef211ca22f20f09fbe69bce77b1bdb
Parents: cc6cc0b
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 14:19:03 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:55 2015 +0100

----------------------------------------------------------------------
 .../org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/eccf873f/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java
index fdefc44..2b45202 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoin.java
@@ -27,6 +27,6 @@ public class QueryIterJoin extends QueryIterJoinBase
 {
     public QueryIterJoin(QueryIterator left, QueryIterator right, ExecutionContext qCxt)
     {
-        super(left, right, JoinType.PLAIN, null, qCxt) ;
+        super(left, right, JoinType.INNER, null, qCxt) ;
     }
 }


[07/20] jena git commit: Fix test data

Posted by an...@apache.org.
Fix test data


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/728542a1
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/728542a1
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/728542a1

Branch: refs/heads/master
Commit: 728542a120b779021315f1fefbf6006dfcf28076
Parents: 13294cb
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 11:46:19 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:12 2015 +0100

----------------------------------------------------------------------
 .../java/org/apache/jena/sparql/engine/ref/TestTableJoin.java | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/728542a1/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java
index b52d532..1a2fbde 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java
@@ -44,13 +44,14 @@ public class TestTableJoin extends BaseTest
     Table zero = new TableEmpty() ;
     Table zeroData = SSE.parseTable("(table)") ;
     Table unitData = SSE.parseTable("(table (row))") ;
+    
     Table data1 = SSE.parseTable("(table (row (?a 1) (?b 2)))") ;
     Table data2 = SSE.parseTable("(table (row (?a 1) (?c 3)) (row (?a 9) (?c 5))  )") ;
     Table data3 = SSE.parseTable("(table (row (?a 1) (?c 3)) (row (?a 1) (?c 4)) (row (?a 9) (?c 5))  )") ;
-    
+
     Table data1J2 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) )") ;
-    Table data1LJ2 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) (row (?a 9) (?c 5)) )") ;
-    Table data2LJ1 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) )") ;
+    Table data1LJ2 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) )") ;
+    Table data2LJ1 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) (row (?a 9) (?c 5)) )") ;
     
     Table data1J3 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) (row (?a 1) (?b 2) (?c 4)) )") ;
     


[17/20] jena git commit: JENA-1023: Easily switchable when use join() and leftJoin().

Posted by an...@apache.org.
JENA-1023: Easily switchable when use join() and leftJoin().


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/03afa9b8
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/03afa9b8
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/03afa9b8

Branch: refs/heads/master
Commit: 03afa9b8fd2e692f747d3a9e6e96606714eb3d6f
Parents: 8f3df5a
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 17:55:40 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:09:37 2015 +0100

----------------------------------------------------------------------
 .../apache/jena/sparql/engine/join/Join.java    | 118 +++++++++++--------
 1 file changed, 70 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/03afa9b8/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
index 4d4523f..5b09544 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
@@ -18,13 +18,11 @@
 
 package org.apache.jena.sparql.engine.join;
 
-import static org.apache.jena.sparql.algebra.JoinType.LEFT ;
-import static org.apache.jena.sparql.algebra.JoinType.INNER ;
-
 import java.util.List ;
 
 import org.apache.jena.atlas.iterator.Iter ;
 import org.apache.jena.atlas.lib.DS ;
+import org.apache.jena.atlas.lib.NotImplemented ;
 import org.apache.jena.sparql.algebra.Algebra ;
 import org.apache.jena.sparql.algebra.JoinType ;
 import org.apache.jena.sparql.algebra.Table ;
@@ -43,11 +41,10 @@ import org.apache.jena.sparql.expr.ExprList ;
 public class Join {
     // See also package org.apache.jena.sparql.engine.index
     // The anti-join code for MINUS
-
-    // *******************
-    // USES OLD TableJoin CODE AT THE MOMENT
-    // *******************
     
+    private final static boolean useNestedLoopJoin = false ;
+    private final static boolean useNestedLoopLeftJoin = false ;
+
     /**
      * Standard entry point to a join of two streams.
      * This is not a substitution/index join.
@@ -58,7 +55,9 @@ public class Join {
      * @return QueryIterator
      */
     public static QueryIterator join(QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
-        return joinWorker(left, right, INNER, null, execCxt) ;
+        if ( useNestedLoopJoin )
+            return nestedLoopJoin(left, right, execCxt) ;
+        return hashJoin(left, right, execCxt) ;
     }
    
     /** Standard entry point to a left join of two streams.
@@ -72,7 +71,10 @@ public class Join {
      * @return QueryIterator
      */
     public static QueryIterator leftJoin(QueryIterator left, QueryIterator right, ExprList conditions, ExecutionContext execCxt) {
-        return joinWorker(left, right, LEFT, conditions, execCxt) ;
+//        if ( useNestedLoopJoin )
+//            return nestedLoopLeftJoin(left, right, conditions, execCxt) ;
+//        return hashLeftJoin(left, right, execCxt) ;
+        return nestedLoopLeftJoin(left, right, conditions, execCxt) ;
     }
    
 
@@ -80,6 +82,8 @@ public class Join {
      * Standard entry point to a join of two streams.
      * This is not a substitution/index join.
      * (See {@link OpExecutor} for streamed execution using substitution).
+     * @deprecated Do not use directly.
+     * 
      * @param left
      * @param right
      * @param joinType
@@ -87,67 +91,85 @@ public class Join {
      * @param execCxt
      * @return QueryIterator
      */
+    @Deprecated
     public static QueryIterator joinWorker(QueryIterator left, QueryIterator right, 
                                            JoinType joinType, ExprList conditions,
                                            ExecutionContext execCxt) {
-        // Catch easy cases.
-        // If left or right is a root, do fast.
-//        if ( right.isEmpty() ) {
-//            if ( joinType == PLAIN ) {
-//                // No rows - no match
-//                left.close() ;
-//                return QueryIterNullIterator.create(execCxt) ;
-//            }
-//            else
-//                // Left join - pass out left rows regardless of conditions.
-//                return left ;
-//        }
-//        
-//        if ( TableUnit.isTableUnit(right) )
-//            return applyConditions(left, conditions, execCxt) ;
-//        return joinWorkerN(left, right, joinType, conditions, execCxt) ;
-        // XXX TEMPORARY!
-        //throw new NotImplemented() ;
-        Table t = TableFactory.create(right) ;
-        return TableJoin.joinWorker(left, t, joinType, conditions, execCxt) ;
+        if ( false ) {
+            // Safe, and slow.
+            switch(joinType) {
+                case INNER: return nestedLoopJoin(left, right, execCxt) ;
+                case LEFT:  return nestedLoopLeftJoin(left, right, conditions, execCxt) ;
+                default:    throw new NotImplemented("JoinType "+joinType+" not implemented") ;
+            }
+        }
+        if ( false ) {
+            // Very safe, and slow.
+            switch(joinType) {
+                case INNER: return nestedLoopJoinBasic(left, right, execCxt) ;
+                case LEFT:  return nestedLoopLeftJoinBasic(left, right, conditions, execCxt) ;
+                default:    throw new NotImplemented("JoinType "+joinType+" not implemented") ;
+            }
+        }
+
+        if ( false ) {
+            // Very safe, well used old code.  And slow.
+            Table t = TableFactory.create(right) ;
+            return TableJoin.joinWorker(left, t, joinType, conditions, execCxt) ;
+        }
+        
+        // Normal.
+        switch(joinType) {
+            case INNER: return join(left, right, execCxt) ;
+            case LEFT:  return leftJoin(left, right, conditions, execCxt) ;
+            default:    throw new NotImplemented("JoinType "+joinType+" not implemented") ;
+        }
     }
-    
-    
+
     /** Inner loop join.
      *  Cancellable.
      * @param left      Left hand side
      * @param right     Right hand side
-     * @param cxt       ExecutionContext
+     * @param execCxt       ExecutionContext
      * @return          QueryIterator
      */ 
-    public static QueryIterator nestedLoopJoin(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
-        return new QueryIterNestedLoopJoin(left, right, cxt) ;
+    public static QueryIterator nestedLoopJoin(QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
+        return new QueryIterNestedLoopJoin(left, right, execCxt) ;
     }
 
     /** Inner loop join.
      *  Cancellable.
      * @param left      Left hand side
      * @param right     Right hand side
-     * @param cxt       ExecutionContext
+     * @param execCxt       ExecutionContext
      * @return          QueryIterator
      */ 
-    public static QueryIterator nestedLoopLeftJoin(QueryIterator left, QueryIterator right, ExprList conditions, ExecutionContext cxt) {
-        return new QueryIterNestedLoopLeftJoin(left, right, conditions, cxt) ;
+    public static QueryIterator nestedLoopLeftJoin(QueryIterator left, QueryIterator right, ExprList conditions, ExecutionContext execCxt) {
+        return new QueryIterNestedLoopLeftJoin(left, right, conditions, execCxt) ;
     }
 
 
-    /** Evaluate a hash join.  This code materializes the left into a probe table
-     * then hash joins from the right.  
-     *  No cancelation, no stats.
+    /** Evaluate using a hash join.
+     * 
+     * @param left      Left hand side
+     * @param right     Right hand side
+     * @param execCxt   ExecutionContext
+     * @return          QueryIterator
+     */
+    public static QueryIterator hashJoin(QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
+        return QueryIterHashJoin.create(left, right, execCxt) ;
+    }
+
+    /** Evaluate using a hash join.
      * 
      * @param joinKey   The key for the probe table.
      * @param left      Left hand side
      * @param right     Right hand side
-     * @param cxt       ExecutionContext
+     * @param execCxt   ExecutionContext
      * @return          QueryIterator
      */
-    public static QueryIterator hashJoin(JoinKey joinKey, QueryIterator left, QueryIterator right, ExecutionContext cxt) {
-        return new QueryIterHashJoin(joinKey, left, right, cxt) ;
+    public static QueryIterator hashJoin(JoinKey joinKey, QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
+        return QueryIterHashJoin.create(joinKey, left, right, execCxt) ;
     }
 
     /** Very simple, materializing version - useful for debugging.
@@ -157,7 +179,7 @@ public class Join {
      * 
      * @see #nestedLoopJoin
      */
-    public static QueryIterator nestedLoopJoinBasic(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
+    public static QueryIterator nestedLoopJoinBasic(QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
         List<Binding> leftRows = Iter.toList(left) ;
         List<Binding> output = DS.list() ;
         for ( ; right.hasNext() ; ) {
@@ -168,14 +190,14 @@ public class Join {
                     output.add(r) ;
             }
         }
-        return new QueryIterPlainWrapper(output.iterator(), cxt) ;
+        return new QueryIterPlainWrapper(output.iterator(), execCxt) ;
     }
 
     /** Very simple, materializing version for leftjoin - useful for debugging.
      *  Builds output early. Materializes right, streams left.
      *  Does <b>not</b> scale. 
      */
-    public static QueryIterator nestedLoopLeftJoinBasic(QueryIterator left, QueryIterator right, ExprList condition, ExecutionContext cxt) {
+    public static QueryIterator nestedLoopLeftJoinBasic(QueryIterator left, QueryIterator right, ExprList condition, ExecutionContext execCxt) {
         // Stream from left, materialize right.
         List<Binding> rightRows = Iter.toList(right) ;
         List<Binding> output = DS.list() ;
@@ -193,8 +215,8 @@ public class Join {
             if ( ! match )
                 output.add(row1) ;
         }
-        QueryIterator qIter = new QueryIterPlainWrapper(output.iterator(), cxt) ;
-        qIter = applyConditions(qIter, condition, cxt) ;
+        QueryIterator qIter = new QueryIterPlainWrapper(output.iterator(), execCxt) ;
+        qIter = applyConditions(qIter, condition, execCxt) ;
         return qIter ;
     }
 


[15/20] jena git commit: Null joinKey if unset.

Posted by an...@apache.org.
Null joinKey if unset.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/8a25b21c
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/8a25b21c
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/8a25b21c

Branch: refs/heads/master
Commit: 8a25b21cd4e24f3b1f3649fc3d66e3616270725a
Parents: 2318191
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 17:42:12 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:09:17 2015 +0100

----------------------------------------------------------------------
 .../org/apache/jena/sparql/engine/join/AbstractTestJoin.java   | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/8a25b21c/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
index 2ed9647..434af09 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
@@ -235,8 +235,10 @@ public abstract class AbstractTestJoin extends Assert {
             joinKey = JoinKey.create(Var.alloc(var)) ;
         }
         else {
-            // No vars in join key.  Legal, albeit silly.
-            joinKey = new JoinKey.Builder().build() ;
+            // No vars in join key.  Allow implementation to decide
+            // if needed.  Join keys are only needed for hash join
+            // (and related algorithms).
+            joinKey = null ;
         }
 
         executeTest(joinKey, left, right, null, tableOut) ;


[18/20] jena git commit: Factory operations to catch simple cases.

Posted by an...@apache.org.
Factory operations to catch simple cases.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/bb9574e2
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/bb9574e2
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/bb9574e2

Branch: refs/heads/master
Commit: bb9574e280cab46c422c98914861dfc039f43e42
Parents: 03afa9b
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 17:56:11 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:09:44 2015 +0100

----------------------------------------------------------------------
 .../sparql/engine/join/QueryIterHashJoin.java   | 57 ++++++++++++++++++--
 1 file changed, 52 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/bb9574e2/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java
index cf8a520..1b4b29a 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/QueryIterHashJoin.java
@@ -19,12 +19,17 @@
 package org.apache.jena.sparql.engine.join;
 
 import java.util.Iterator ;
+import java.util.List ;
 
+import org.apache.jena.atlas.iterator.Iter ;
 import org.apache.jena.sparql.algebra.Algebra ;
+import org.apache.jena.sparql.core.Var ;
 import org.apache.jena.sparql.engine.ExecutionContext ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.binding.Binding ;
 import org.apache.jena.sparql.engine.iterator.QueryIter2 ;
+import org.apache.jena.sparql.engine.iterator.QueryIterNullIterator ;
+import org.apache.jena.sparql.engine.iterator.QueryIterPeek ;
 
 /** Hash join.  This code materializes the left into a probe table
  * then hash joins from the right.
@@ -49,8 +54,53 @@ public class QueryIterHashJoin extends QueryIter2 {
     private Binding slot = null ;
     private boolean finished = false ; 
 
-    public QueryIterHashJoin(JoinKey joinKey, QueryIterator left, QueryIterator right, ExecutionContext cxt) {
-        super(left, right, cxt) ;
+    /**
+     * Create a hashjoin QueryIterator.
+     * @param joinKey  Join key - if null, one is guessed by snooping the input QueryIterators
+     * @param left
+     * @param right
+     * @param execCxt
+     * @return QueryIterator
+     */
+    public static QueryIterator create(JoinKey joinKey, QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
+        // Easy cases.
+        if ( ! left.hasNext() || ! right.hasNext() ) {
+            left.close() ;
+            right.close() ;
+            return QueryIterNullIterator.create(execCxt) ;
+        }
+        return new QueryIterHashJoin(joinKey, left, right, execCxt) ; 
+    }
+    
+    /**
+     * Create a hashjoin QueryIterator.
+     * @param left
+     * @param right
+     * @param execCxt
+     * @return QueryIterator
+     */
+ 
+    public static QueryIterator create(QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
+        return create(null, left, right, execCxt) ;
+    }
+    
+    private QueryIterHashJoin(JoinKey joinKey, QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
+        super(left, right, execCxt) ;
+        
+        if ( joinKey == null ) {
+            QueryIterPeek pLeft = QueryIterPeek.create(left, execCxt) ;
+            QueryIterPeek pRight = QueryIterPeek.create(right, execCxt) ;
+            
+            Binding bLeft = pLeft.peek() ;
+            Binding bRight = pRight.peek() ;
+            
+            List<Var> varsLeft = Iter.toList(bLeft.vars()) ;
+            List<Var> varsRight = Iter.toList(bRight.vars()) ;
+            joinKey = JoinKey.create(varsLeft, varsRight) ;
+            left = pLeft ;
+            right = pRight ;
+        }
+        
         this.joinKey = joinKey ;
         this.iterRight = right ;
         this.hashTable = new HashProbeTable(joinKey) ;
@@ -119,9 +169,6 @@ public class QueryIterHashJoin extends QueryIter2 {
             if (r != null) {
                 s_countResults ++ ;
                 return r ;
-            } else {
-                // XXX LeftJoin
-                // return rowLeft ;
             }
         }
     }        


[20/20] jena git commit: JENA-1023: Use "Join" for OpJoin and OpLeftJoin.

Posted by an...@apache.org.
JENA-1023: Use "Join" for OpJoin and OpLeftJoin.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/668db14a
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/668db14a
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/668db14a

Branch: refs/heads/master
Commit: 668db14a495933ad43f9c484c1dbf01a9b419bbd
Parents: a101245
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 17:56:51 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:10:04 2015 +0100

----------------------------------------------------------------------
 .../main/java/org/apache/jena/sparql/engine/main/OpExecutor.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/668db14a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java
index 801a48e..3929975 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java
@@ -214,7 +214,7 @@ public class OpExecutor
 
             QueryIterator left = exec(opJoin.getLeft(), qIter1) ;
             QueryIterator right = exec(opJoin.getRight(), qIter2) ;
-            QueryIterator qIter = new QueryIterJoin(left, right, execCxt) ;
+            QueryIterator qIter = Join.join(left, right, execCxt) ;
             return qIter ;
         }
         QueryIterator left = exec(opJoin.getLeft(), input) ;
@@ -329,7 +329,7 @@ public class OpExecutor
             return opTable.getTable().iterator(execCxt) ;
         }
         QueryIterator qIterT = opTable.getTable().iterator(execCxt) ;
-        QueryIterator qIter = new QueryIterJoin(input, qIterT, execCxt) ;
+        QueryIterator qIter = Join.join(input, qIterT, execCxt) ;
         return qIter ;
     }
 


[03/20] jena git commit: Move join test package to aling to main. Start leftjoin tests.

Posted by an...@apache.org.
Move join test package to aling to main.  Start leftjoin tests.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/d4da1a5a
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/d4da1a5a
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/d4da1a5a

Branch: refs/heads/master
Commit: d4da1a5a890073d2bd8114e8e611ddaf88eae8fd
Parents: 0b060c8
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 10:22:57 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:11 2015 +0100

----------------------------------------------------------------------
 .../java/org/apache/jena/sparql/TC_General.java |   2 +-
 .../apache/jena/sparql/engine/TS_Engine.java    |   3 +-
 .../apache/jena/sparql/engine/TestTableLib.java | 119 ----------
 .../engine/join/AbstractTestInnerJoin.java      | 170 +++++++++++++
 .../sparql/engine/join/AbstractTestJoin.java    | 101 ++++++++
 .../engine/join/AbstractTestLeftJoin.java       | 170 +++++++++++++
 .../apache/jena/sparql/engine/join/TS_Join.java |  34 +++
 .../jena/sparql/engine/join/TestHashJoin.java   |  38 +++
 .../sparql/engine/join/TestJoinNestedLoop.java  |  34 +++
 .../engine/join/TestJoinNestedLoopSimple.java   |  34 +++
 .../jena/sparql/engine/join/TestJoinSimple.java |  34 +++
 .../sparql/engine/join/TestLeftJoinSimple.java  |  34 +++
 .../jena/sparql/join/AbstractTestJoin.java      | 236 -------------------
 .../org/apache/jena/sparql/join/TS_Join.java    |  34 ---
 .../apache/jena/sparql/join/TestHashJoin.java   |  38 ---
 .../jena/sparql/join/TestJoinNestedLoop.java    |  34 ---
 .../sparql/join/TestJoinNestedLoopSimple.java   |  34 ---
 .../apache/jena/sparql/join/TestJoinSimple.java |  34 ---
 18 files changed, 652 insertions(+), 531 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java b/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java
index 7c67a5d..429387e 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/TC_General.java
@@ -25,12 +25,12 @@ import org.apache.jena.sparql.algebra.optimize.TS_Optimization ;
 import org.apache.jena.sparql.api.TS_API ;
 import org.apache.jena.sparql.core.TS_Core ;
 import org.apache.jena.sparql.engine.TS_Engine ;
+import org.apache.jena.sparql.engine.join.TS_Join ;
 import org.apache.jena.sparql.expr.E_Function ;
 import org.apache.jena.sparql.expr.NodeValue ;
 import org.apache.jena.sparql.expr.TS_Expr ;
 import org.apache.jena.sparql.function.user.TS_UserFunctions ;
 import org.apache.jena.sparql.graph.TS_Graph ;
-import org.apache.jena.sparql.join.TS_Join ;
 import org.apache.jena.sparql.lang.TS_Lang ;
 import org.apache.jena.sparql.modify.TS_Update ;
 import org.apache.jena.sparql.negation.TS_Negation ;

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/TS_Engine.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/TS_Engine.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/TS_Engine.java
index 889acf9..d7b386c 100644
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/TS_Engine.java
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/TS_Engine.java
@@ -22,13 +22,14 @@ import org.apache.jena.sparql.engine.binding.TestBindingStreams ;
 import org.apache.jena.sparql.engine.http.TestQueryEngineHTTP ;
 import org.apache.jena.sparql.engine.http.TestService ;
 import org.apache.jena.sparql.engine.iterator.TS_QueryIterators ;
+import org.apache.jena.sparql.engine.ref.TestTableJoin ;
 import org.junit.runner.RunWith ;
 import org.junit.runners.Suite ;
 
 @RunWith(Suite.class)
 @Suite.SuiteClasses( {
         TestBindingStreams.class
-      , TestTableLib.class
+      , TestTableJoin.class
       , TS_QueryIterators.class
       , TestService.class
       , TestQueryEngineHTTP.class

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/TestTableLib.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/TestTableLib.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/TestTableLib.java
deleted file mode 100644
index 88f8cf0..0000000
--- a/jena-arq/src/test/java/org/apache/jena/sparql/engine/TestTableLib.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.engine;
-
-import java.util.HashSet ;
-import java.util.Set ;
-
-import org.apache.jena.atlas.junit.BaseTest ;
-import org.apache.jena.query.ARQ ;
-import org.apache.jena.query.ResultSet ;
-import org.apache.jena.query.ResultSetFactory ;
-import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.algebra.table.TableEmpty ;
-import org.apache.jena.sparql.algebra.table.TableN ;
-import org.apache.jena.sparql.algebra.table.TableUnit ;
-import org.apache.jena.sparql.engine.ExecutionContext ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-import org.apache.jena.sparql.engine.TableJoin ;
-import org.apache.jena.sparql.expr.ExprList ;
-import org.apache.jena.sparql.resultset.ResultSetCompare ;
-import org.apache.jena.sparql.sse.SSE ;
-import org.junit.Test ;
-
-public class TestTableLib extends BaseTest
-{
-    Table unit = new TableUnit() ; 
-    Table zero = new TableEmpty() ;
-    Table zeroData = SSE.parseTable("(table)") ;
-    Table unitData = SSE.parseTable("(table (row))") ;
-    Table data1 = SSE.parseTable("(table (row (?a 1) (?b 2)))") ;
-    Table data2 = SSE.parseTable("(table (row (?a 1) (?c 3)) (row (?a 9) (?c 5))  )") ;
-    Table data3 = SSE.parseTable("(table (row (?a 1) (?c 3)) (row (?a 1) (?c 4)) (row (?a 9) (?c 5))  )") ;
-    
-    Table data1J2 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) )") ;
-    Table data1LJ2 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) (row (?a 9) (?c 5)) )") ;
-    Table data2LJ1 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) )") ;
-    
-    Table data1J3 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) (row (?a 1) (?b 2) (?c 4)) )") ;
-    
-    @Test public void table_01() { test(unit, zero, true, null, zero) ; }
-    @Test public void table_02() { test(zero, unit, true, null, zero) ; }
-    @Test public void table_03() { test(unit, zero, false, null, unit) ; }
-    @Test public void table_04() { test(zero, unit, false, null, zero) ; }
-    // Same again but with non-special tables.
-    @Test public void table_05() { test(unitData, zeroData, true, null, zeroData) ; }
-    @Test public void table_06() { test(zeroData, unitData, true, null, zeroData) ; }
-    @Test public void table_07() { test(unitData, zeroData, false, null, unitData) ; }
-    @Test public void table_08() { test(zeroData, unitData, false, null, zeroData) ; }
-
-    @Test public void table_10() { test(data1, zero, true, null, zero) ; }
-    @Test public void table_11() { test(zero, data1, true, null, zero) ; }
-    @Test public void table_12() { test(data1, zero, false, null, data1) ; }
-    @Test public void table_13() { test(zero, data1, false, null, zero) ; }
-    
-    @Test public void table_14() { test(data1, zeroData, true, null, zeroData) ; }
-    @Test public void table_15() { test(zeroData, data1, true, null, zeroData) ; }
-    @Test public void table_16() { test(data1, zeroData, false, null, data1) ; }
-    @Test public void table_17() { test(zeroData, data1, false, null, zeroData) ; }
-
-    @Test public void table_18() { test(data2, unitData, true, null, data2) ; }
-    @Test public void table_19() { test(unitData, data2, true, null, data2) ; }
-    
-    @Test public void table_20() { test(data1, data2, true, null, data1J2) ; }
-    @Test public void table_21() { test(data2, data1, true, null, data1J2) ; }
-    @Test public void table_22() { test(data1, data2, false, null, data1LJ2) ; }
-    @Test public void table_23() { test(data2, data1, false, null, data2LJ1) ; }
-    
-    @Test public void table_24() { test(data1, data3, true, null, data1J3) ; }
-    @Test public void table_25() { test(data3, data1, true, null, data1J3) ; }
-
-    private void test(Table left, Table right, boolean normalJoin, ExprList exprs, Table expected) {
-        ExecutionContext execCxt = new ExecutionContext(ARQ.getContext(), null, null, null) ;
-        QueryIterator leftIter = left.iterator(execCxt) ;
-        QueryIterator qIter = normalJoin 
-            ? TableJoin.join(leftIter, right, exprs, execCxt)
-            : TableJoin.leftJoin(leftIter, right, exprs, execCxt) ;
-        
-            // Order issues
-            
-        Set<String> vars1 = new HashSet<>() ;
-        vars1.addAll(left.getVarNames()) ;
-        vars1.addAll(right.getVarNames()) ;
-        
-        TableN results = new TableN(qIter) ;
-        boolean b = TableCompare.equalsByTerm(expected, results) ;
-        if ( !b ) {
-            System.out.println("** Expected") ;
-            System.out.println(expected) ;
-            System.out.println("** Actual") ;
-            System.out.println(results) ;
-        }
-        assertTrue(b) ;
-    }
-    
-    static class TableCompare {
-        public static boolean equalsByTerm(Table table1, Table table2) {
-            ResultSet rs1 = ResultSetFactory.create(table1.iterator(null), table1.getVarNames()) ;
-            ResultSet rs2 = ResultSetFactory.create(table2.iterator(null), table2.getVarNames()) ;
-            return ResultSetCompare.equalsByTerm(rs1, rs2) ;
-        }
-    }
-        
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java
new file mode 100644
index 0000000..f5a1bdc
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestInnerJoin.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.core.Var ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.junit.Test ;
+
+/** Tests for inner/equi joins */ 
+public abstract class AbstractTestInnerJoin extends AbstractTestJoin {
+    
+    @Override
+    protected void executeTest(JoinKey joinKey, Table left, Table right, Table expectedResults) {
+        executeTestJoin("1", joinKey, left, right, expectedResults) ;
+        executeTestJoin("2", joinKey, right, left, expectedResults) ;
+    }
+    
+    static Var var_a = Var.alloc("a") ; 
+    static Var var_b = Var.alloc("b") ; 
+    static Var var_c = Var.alloc("c") ; 
+    static Var var_d = Var.alloc("d") ; 
+
+    static Table table0() { return parseTableInt("(table)") ; } 
+
+    // Table of one row and no colums.
+    static Table table1() { 
+        return parseTableInt("(table (row))") ; }
+
+    static Table tableD1() { 
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?b 2))",
+                             "   (row (?a 1) (?b 3))",
+                             "   (row (?a 1) (?b 2))",
+            ")") ;
+    }
+
+    static Table tableD2() { 
+        return parseTableInt("(table", 
+                             "   (row (?a 0) (?d 8))",
+                             "   (row (?a 1) (?c 9))",
+            ")") ;
+    }
+
+    static Table tableD3() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 9) (?b 2))",
+                             "   (row (?a 1) (?c 9) (?b 3))",
+                             "   (row (?a 1) (?c 9) (?b 2))",
+            ")") ;
+    }
+
+    static Table tableD4() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?b 2))",
+                             "   (row (?a 1) (?b 3))",
+                             "   (row (?a 4) (?b 4))",
+                             "   (row (?a 4) (?b 5))",
+            ")") ;
+    }
+
+    static Table tableD5() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4))",
+                             "   (row (?a 4) (?c 5))",
+                             "   (row (?a 6) (?c 5))",
+            ")") ;
+    }
+
+    static Table tableD6() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 2))",
+                             "   (row (?a 1) (?c 3))",
+                             "   (row (?a 4) (?c 4))",
+                             "   (row (?a 4) (?c 5))",
+            ")") ;
+    }
+
+    static Table tableD4x5() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+            ")") ;
+    }
+
+    static Table tableD4x6() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 2) (?b 2))",
+                             "   (row (?a 1) (?c 2) (?b 3))",
+                             "   (row (?a 1) (?c 3) (?b 2))",
+                             "   (row (?a 1) (?c 3) (?b 3))",
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+            ")") ;
+    }
+
+    // Disjoint.
+    static Table tableD8() {
+        return parseTableInt("(table",
+                             "  (row (?x 10))",
+                             "  (row (?z 11))",
+            ")") ; 
+    }
+
+    // Table8 crossproduct table2
+    static Table tableD8x2() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?d 8) (?z 11))",
+                             "  (row (?a 0) (?d 8) (?x 10))",
+                             "  (row (?a 1) (?c 9) (?z 11))",
+                             "  (row (?a 1) (?c 9) (?x 10))",
+            ")") ;
+    }
+
+    @Test public void join_00()  { testJoin("a", table0(), table0(), table0()) ; }
+    @Test public void join_00a() { testJoin("a", table1(), table0(), table0()) ; }
+    @Test public void join_00b() { testJoin("a", tableD1(), table1(), tableD1()) ; }
+    @Test public void join_00c() { testJoin("z", tableD1(), table1(), tableD1()) ; }
+
+    @Test public void join_01() { testJoin("a", table0(), tableD2(), table0()) ; }
+    @Test public void join_02() { testJoin("a", tableD1(), table0(), table0()) ; }
+    @Test public void join_03() { testJoin("a", tableD1(), tableD2(), tableD3()) ; }
+
+    // Identity.
+    @Test public void join_04() { testJoin("a", tableD2(), table1(), tableD2()) ; }
+    @Test public void join_05() { testJoin("a", table1(), tableD2(), tableD2()) ; }
+    // Identity, keymiss
+    @Test public void join_06() { testJoin("z", table1(), tableD2(), tableD2()) ; }
+    @Test public void join_07() { testJoin("z", tableD2(), table1(), tableD2()) ; }
+
+    @Test public void join_08() { testJoin("a", tableD4(), tableD5(), tableD4x5()) ; }
+    @Test public void join_09() { testJoin("a", tableD5(), tableD4(), tableD4x5()) ; }
+
+    @Test public void join_10() { testJoin("a", tableD4(), tableD6(), tableD4x6()) ; }
+    @Test public void join_11() { testJoin("a", tableD6(), tableD4(), tableD4x6()) ; }
+
+    // Not the right join key - should still work albeit less efficiently.
+    @Test public void join_12() { testJoin("z", tableD1(), tableD2(), tableD3()) ; }
+    @Test public void join_13() { testJoin("z", tableD2(), tableD1(), tableD3()) ; }
+
+    // No key.
+    @Test public void join_14() { testJoin(null, tableD1(), tableD2(), tableD3()) ; }
+
+
+    // Disjoint tables.
+    @Test public void join_disjoint_01() { testJoin("a", tableD2(), tableD8(), tableD8x2()) ; }
+    @Test public void join_disjoint_02() { testJoin("z", tableD2(), tableD8(), tableD8x2()) ; }
+}
+
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
new file mode 100644
index 0000000..b48c66b
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestJoin.java
@@ -0,0 +1,101 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import java.util.List ;
+
+import org.apache.jena.atlas.iterator.Iter ;
+import org.apache.jena.atlas.lib.StrUtils ;
+import org.apache.jena.query.ResultSet ;
+import org.apache.jena.query.ResultSetFactory ;
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.algebra.TableFactory ;
+import org.apache.jena.sparql.core.Var ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.binding.Binding ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.resultset.ResultSetCompare ;
+import org.apache.jena.sparql.sse.SSE ;
+import org.junit.Assert ;
+
+/** Tests for inner/equi joins */ 
+public abstract class AbstractTestJoin extends Assert {
+    protected static Table parseTableInt(String... strings) {
+        String x = StrUtils.strjoinNL(strings) ;
+        return SSE.parseTable(x) ;
+    }
+
+    protected void testJoin(String var, Table left, Table right, Table tableOut) {
+        JoinKey joinKey ;
+        if ( var != null ) {
+            if ( var.startsWith("?") )
+                var = var.substring(1) ;
+            joinKey = JoinKey.create(Var.alloc(var)) ;
+        }
+        else {
+            // No vars in join key.  Legal, albeit silly.
+            joinKey = new JoinKey.Builder().build() ;
+        }
+
+        executeTest(joinKey, left, right, tableOut) ;
+    }
+
+    protected abstract void executeTest(JoinKey joinKey, Table left, Table right, Table expectedResults) ;
+    
+    
+    private List<Binding> toList(Table table) {
+        return Iter.toList(table.rows()) ;
+    }
+
+    protected void executeTestJoin(String num, JoinKey joinKey, Table left, Table right, Table expectedResults) {
+        Table x1 = joinMaterialize(joinKey, left, right) ;
+        assertNotNull("Null table from join ("+num+")", x1) ;
+        check("Results not equal ("+num+")", joinKey, left, right, expectedResults, x1) ;
+    }
+
+    private Table joinMaterialize(JoinKey joinKey, Table left, Table right) {
+        QueryIterator qIter = join(joinKey, left , right) ;
+        return TableFactory.create(qIter) ;
+    }
+
+    public abstract QueryIterator join(JoinKey joinKey , Table left , Table right) ;
+
+    private static void check(String msg, JoinKey joinKey, Table left, Table right, Table expected, Table actual) {
+        boolean b = equalTables(expected, actual) ;
+        if ( ! b ) {
+            System.out.println("Joinkey:  "+joinKey) ;
+            System.out.println("Left:     \n"+left) ;
+            System.out.println("Right:    \n"+right) ;
+            System.out.println("Expected: \n"+expected) ;
+            System.out.println("Actual:   \n"+actual) ;
+            System.out.println() ;
+        }
+
+        assertTrue(msg, b) ;
+    }
+
+    private static boolean equalTables(Table table1, Table table2) {
+        ResultSet rs1 =  ResultSetFactory.create(table1.iterator(null), table1.getVarNames()) ;
+        ResultSet rs2 =  ResultSetFactory.create(table2.iterator(null), table2.getVarNames()) ;
+        return ResultSetCompare.equalsByTerm(rs1, rs2) ;
+    }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java
new file mode 100644
index 0000000..57756ff
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/AbstractTestLeftJoin.java
@@ -0,0 +1,170 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.core.Var ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.junit.Test ;
+
+public abstract class AbstractTestLeftJoin extends AbstractTestJoin {
+    @Override
+    protected void executeTest(JoinKey joinKey, Table left, Table right, Table expectedResults) {
+        executeTestJoin("LJ", joinKey, left, right, expectedResults) ;
+    }
+    
+    static Var var_a = Var.alloc("a") ; 
+    static Var var_b = Var.alloc("b") ; 
+    static Var var_c = Var.alloc("c") ; 
+    static Var var_d = Var.alloc("d") ; 
+
+    static Table table0() { return parseTableInt("(table)") ; } 
+
+    // Table of one row and no colums.
+    static Table table1() { 
+        return parseTableInt("(table (row))") ; }
+
+    static Table tableD1() { 
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?b 2))",
+                             "   (row (?a 1) (?b 3))",
+                             "   (row (?a 1) (?b 2))",
+            ")") ;
+    }
+
+    static Table tableD2() { 
+        return parseTableInt("(table", 
+                             "   (row (?a 0) (?d 8))",
+                             "   (row (?a 1) (?c 9))",
+            ")") ;
+    }
+
+    static Table tableD3() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 9) (?b 2))",
+                             "   (row (?a 1) (?c 9) (?b 3))",
+                             "   (row (?a 1) (?c 9) (?b 2))",
+            ")") ;
+    }
+
+    static Table tableD4() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?b 2))",
+                             "   (row (?a 1) (?b 3))",
+                             "   (row (?a 4) (?b 4))",
+                             "   (row (?a 4) (?b 5))",
+            ")") ;
+    }
+
+    static Table tableD5() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4))",
+                             "   (row (?a 4) (?c 5))",
+                             "   (row (?a 6) (?c 5))",
+            ")") ;
+    }
+
+    static Table tableD6() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 2))",
+                             "   (row (?a 1) (?c 3))",
+                             "   (row (?a 4) (?c 4))",
+                             "   (row (?a 4) (?c 5))",
+            ")") ;
+    }
+
+    static Table tableD4x5() {
+        return parseTableInt("(table", 
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+            ")") ;
+    }
+
+    static Table tableD4x6() {
+        return parseTableInt("(table", 
+                             "   (row (?a 1) (?c 2) (?b 2))",
+                             "   (row (?a 1) (?c 2) (?b 3))",
+                             "   (row (?a 1) (?c 3) (?b 2))",
+                             "   (row (?a 1) (?c 3) (?b 3))",
+                             "   (row (?a 4) (?c 4) (?b 4))",
+                             "   (row (?a 4) (?c 4) (?b 5))",
+                             "   (row (?a 4) (?c 5) (?b 4))",
+                             "   (row (?a 4) (?c 5) (?b 5))",
+            ")") ;
+    }
+
+    // Disjoint.
+    static Table tableD8() {
+        return parseTableInt("(table",
+                             "  (row (?x 10))",
+                             "  (row (?z 11))",
+            ")") ; 
+    }
+
+    // Table8 crossproduct table2
+    static Table tableD8x2() {
+        return parseTableInt("(table",
+                             "  (row (?a 0) (?d 8) (?z 11))",
+                             "  (row (?a 0) (?d 8) (?x 10))",
+                             "  (row (?a 1) (?c 9) (?z 11))",
+                             "  (row (?a 1) (?c 9) (?x 10))",
+            ")") ;
+    }
+
+    // XXX And conditions.
+    
+    @Test public void leftjoin_00()  { testJoin("a", table0(), table0(), table0()) ; }
+    @Test public void leftjoin_00a() { testJoin("a", table1(), table0(), table0()) ; }
+    @Test public void leftjoin_00b() { testJoin("a", tableD1(), table1(), tableD1()) ; }
+    @Test public void leftjoin_00c() { testJoin("z", tableD1(), table1(), tableD1()) ; }
+
+    @Test public void leftjoin_01() { testJoin("a", table0(), tableD2(), table0()) ; }
+    @Test public void leftjoin_02() { testJoin("a", tableD1(), table0(), tableD1()) ; }
+    @Test public void leftjoin_03() { testJoin("a", tableD1(), tableD2(), tableD3()) ; }
+
+    // Identity.
+    @Test public void leftjoin_04() { testJoin("a", tableD2(), table1(), tableD2()) ; }
+    @Test public void leftjoin_05() { testJoin("a", table1(), tableD2(), tableD2()) ; }
+    // Identity, keymiss
+    @Test public void leftjoin_06() { testJoin("z", table1(), tableD2(), tableD2()) ; }
+    @Test public void leftjoin_07() { testJoin("z", tableD2(), table1(), tableD2()) ; }
+
+    @Test public void leftjoin_08() { testJoin("a", tableD4(), tableD5(), tableD4x5()) ; }
+    @Test public void leftjoin_09() { testJoin("a", tableD5(), tableD4(), tableD4x5()) ; }
+
+    @Test public void leftjoin_10() { testJoin("a", tableD4(), tableD6(), tableD4x6()) ; }
+    @Test public void leftjoin_11() { testJoin("a", tableD6(), tableD4(), tableD4x6()) ; }
+
+    // Not the right join key - should still work albeit less efficiently.
+    @Test public void leftjoin_12() { testJoin("z", tableD1(), tableD2(), tableD3()) ; }
+    @Test public void leftjoin_13() { testJoin("z", tableD2(), tableD1(), tableD3()) ; }
+
+    // No key.
+    @Test public void leftjoin_14() { testJoin(null, tableD1(), tableD2(), tableD3()) ; }
+
+
+    // Disjoint tables.
+    @Test public void leftjoin_disjoint_01() { testJoin("a", tableD2(), tableD8(), tableD8x2()) ; }
+    @Test public void leftjoin_disjoint_02() { testJoin("z", tableD2(), tableD8(), tableD8x2()) ; }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java
new file mode 100644
index 0000000..69b5fb4
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TS_Join.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.junit.runner.RunWith ;
+import org.junit.runners.Suite ;
+import org.junit.runners.Suite.SuiteClasses ;
+
+@RunWith(Suite.class)
+@SuiteClasses( {
+    TestJoinSimple.class
+    , TestJoinNestedLoopSimple.class    // Real simple materializing version.
+    , TestJoinNestedLoop.class
+    , TestHashJoin.class
+})
+
+public class TS_Join { }
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java
new file mode 100644
index 0000000..4a4bd6f
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestHashJoin.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import java.util.Iterator ;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.binding.Binding ;
+import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
+import org.apache.jena.sparql.engine.join.Join ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+
+public class TestHashJoin extends AbstractTestInnerJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        Iterator<Binding> data = Join.hashJoin(joinKey, left.iterator(null), right.iterator(null), null) ;
+        QueryIterator qIter = new QueryIterPlainWrapper(data) ;
+        return qIter ;
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java
new file mode 100644
index 0000000..c844ea2
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoop.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.join.Join ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+
+public class TestJoinNestedLoop extends AbstractTestInnerJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        return Join.innerLoopJoin(left.iterator(null), right.iterator(null), null) ;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java
new file mode 100644
index 0000000..0d9db54
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinNestedLoopSimple.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.join.Join ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+
+public class TestJoinNestedLoopSimple extends AbstractTestInnerJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        return Join.innerLoopJoinBasic(left.iterator(null), right.iterator(null), null) ;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java
new file mode 100644
index 0000000..d091168
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestJoinSimple.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.engine.ref.TableJoin ;
+
+public class TestJoinSimple extends AbstractTestInnerJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        return TableJoin.join(left.iterator(null), right, null, null) ;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java
new file mode 100644
index 0000000..041dfe5
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/join/TestLeftJoinSimple.java
@@ -0,0 +1,34 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.join;
+
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.join.JoinKey ;
+import org.apache.jena.sparql.engine.ref.TableJoin ;
+
+public class TestLeftJoinSimple extends AbstractTestLeftJoin {
+
+    @Override
+    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
+        return TableJoin.leftJoin(left.iterator(null), right, null, null) ;
+    }
+
+}
+

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/join/AbstractTestJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/AbstractTestJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/AbstractTestJoin.java
deleted file mode 100644
index a15d3e3..0000000
--- a/jena-arq/src/test/java/org/apache/jena/sparql/join/AbstractTestJoin.java
+++ /dev/null
@@ -1,236 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.join;
-
-import java.util.List ;
-
-import org.apache.jena.atlas.iterator.Iter ;
-import org.apache.jena.atlas.lib.StrUtils ;
-import org.apache.jena.query.ResultSet ;
-import org.apache.jena.query.ResultSetFactory ;
-import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.algebra.TableFactory ;
-import org.apache.jena.sparql.core.Var ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-import org.apache.jena.sparql.engine.binding.Binding ;
-import org.apache.jena.sparql.engine.join.JoinKey ;
-import org.apache.jena.sparql.resultset.ResultSetCompare ;
-import org.apache.jena.sparql.sse.SSE ;
-import org.junit.Assert ;
-import org.junit.Test ;
-
-public abstract class AbstractTestJoin extends Assert {
-    static Var var_a = Var.alloc("a") ; 
-    static Var var_b = Var.alloc("b") ; 
-    static Var var_c = Var.alloc("c") ; 
-    static Var var_d = Var.alloc("d") ; 
-
-    static Table table0() { return parseTableInt("(table)") ; } 
-
-    // For Mere, these must be in sort-joinkey order.
-
-    private static Table parseTableInt(String... strings) {
-        String x = StrUtils.strjoinNL(strings) ;
-        return SSE.parseTable(x) ;
-    }
-
-    // Table of one row and no colums.
-    static Table table1() { 
-        return parseTableInt("(table (row))") ; }
-
-    static Table tableD1() { 
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?b 2))",
-                             "   (row (?a 1) (?b 3))",
-                             "   (row (?a 1) (?b 2))",
-            ")") ;
-    }
-
-    static Table tableD2() { 
-        return parseTableInt("(table", 
-                             "   (row (?a 0) (?d 8))",
-                             "   (row (?a 1) (?c 9))",
-            ")") ;
-    }
-
-    static Table tableD3() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 9) (?b 2))",
-                             "   (row (?a 1) (?c 9) (?b 3))",
-                             "   (row (?a 1) (?c 9) (?b 2))",
-            ")") ;
-    }
-
-    static Table tableD4() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?b 2))",
-                             "   (row (?a 1) (?b 3))",
-                             "   (row (?a 4) (?b 4))",
-                             "   (row (?a 4) (?b 5))",
-            ")") ;
-    }
-
-    static Table tableD5() {
-        return parseTableInt("(table", 
-                             "   (row (?a 4) (?c 4))",
-                             "   (row (?a 4) (?c 5))",
-                             "   (row (?a 6) (?c 5))",
-            ")") ;
-    }
-
-    static Table tableD6() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 2))",
-                             "   (row (?a 1) (?c 3))",
-                             "   (row (?a 4) (?c 4))",
-                             "   (row (?a 4) (?c 5))",
-            ")") ;
-    }
-
-    static Table tableD4x5() {
-        return parseTableInt("(table", 
-                             "   (row (?a 4) (?c 4) (?b 4))",
-                             "   (row (?a 4) (?c 4) (?b 5))",
-                             "   (row (?a 4) (?c 5) (?b 4))",
-                             "   (row (?a 4) (?c 5) (?b 5))",
-            ")") ;
-    }
-
-    static Table tableD4x6() {
-        return parseTableInt("(table", 
-                             "   (row (?a 1) (?c 2) (?b 2))",
-                             "   (row (?a 1) (?c 2) (?b 3))",
-                             "   (row (?a 1) (?c 3) (?b 2))",
-                             "   (row (?a 1) (?c 3) (?b 3))",
-                             "   (row (?a 4) (?c 4) (?b 4))",
-                             "   (row (?a 4) (?c 4) (?b 5))",
-                             "   (row (?a 4) (?c 5) (?b 4))",
-                             "   (row (?a 4) (?c 5) (?b 5))",
-            ")") ;
-    }
-
-    // Disjoint.
-    static Table tableD8() {
-        return parseTableInt("(table",
-                             "  (row (?x 10))",
-                             "  (row (?z 11))",
-            ")") ; 
-    }
-
-    // Table8 crossproduct table2
-    static Table tableD8x2() {
-        return parseTableInt("(table",
-                             "  (row (?a 0) (?d 8) (?z 11))",
-                             "  (row (?a 0) (?d 8) (?x 10))",
-                             "  (row (?a 1) (?c 9) (?z 11))",
-                             "  (row (?a 1) (?c 9) (?x 10))",
-            ")") ;
-    }
-
-    @Test public void join_00()  { testJoin("a", table0(), table0(), table0()) ; }
-    @Test public void join_00a() { testJoin("a", table1(), table0(), table0()) ; }
-    @Test public void join_00b() { testJoin("a", tableD1(), table1(), tableD1()) ; }
-    @Test public void join_00c() { testJoin("z", tableD1(), table1(), tableD1()) ; }
-
-    @Test public void join_01() { testJoin("a", table0(), tableD2(), table0()) ; }
-    @Test public void join_02() { testJoin("a", tableD1(), table0(), table0()) ; }
-    @Test public void join_03() { testJoin("a", tableD1(), tableD2(), tableD3()) ; }
-
-    // Identity.
-    @Test public void join_04() { testJoin("a", tableD2(), table1(), tableD2()) ; }
-    @Test public void join_05() { testJoin("a", table1(), tableD2(), tableD2()) ; }
-    // Identity, keymiss
-    @Test public void join_06() { testJoin("z", table1(), tableD2(), tableD2()) ; }
-    @Test public void join_07() { testJoin("z", tableD2(), table1(), tableD2()) ; }
-
-    @Test public void join_08() { testJoin("a", tableD4(), tableD5(), tableD4x5()) ; }
-    @Test public void join_09() { testJoin("a", tableD5(), tableD4(), tableD4x5()) ; }
-
-    @Test public void join_10() { testJoin("a", tableD4(), tableD6(), tableD4x6()) ; }
-    @Test public void join_11() { testJoin("a", tableD6(), tableD4(), tableD4x6()) ; }
-
-    // Not the right join key - should still work albeit less efficiently.
-    @Test public void join_12() { testJoin("z", tableD1(), tableD2(), tableD3()) ; }
-    @Test public void join_13() { testJoin("z", tableD2(), tableD1(), tableD3()) ; }
-
-    // No key.
-    @Test public void join_14() { testJoin(null, tableD1(), tableD2(), tableD3()) ; }
-
-
-    // Disjoint tables.
-    @Test public void join_disjoint_01() { testJoin("a", tableD2(), tableD8(), tableD8x2()) ; }
-    @Test public void join_disjoint_02() { testJoin("z", tableD2(), tableD8(), tableD8x2()) ; }
-
-    private void testJoin(String var, Table left, Table right, Table tableOut) {
-        JoinKey joinKey ;
-        if ( var != null ) {
-            if ( var.startsWith("?") )
-                var = var.substring(1) ;
-            joinKey = JoinKey.create(Var.alloc(var)) ;
-        }
-        else {
-            // No vars in join key.  Legal, albeit silly.
-            joinKey = new JoinKey.Builder().build() ;
-        }
-
-        testJoin1("1", joinKey, left, right, tableOut) ;
-        // Commumute
-        testJoin1("2", joinKey, right, left, tableOut) ;
-    }
-
-    private List<Binding> toList(Table table) {
-        return Iter.toList(table.rows()) ;
-    }
-
-    private void testJoin1(String num, JoinKey joinKey, Table left, Table right, Table expectedResults) {
-        Table x1 = joinMaterialize(joinKey, left, right) ;
-        assertNotNull("Null table from join ("+num+")", x1) ;
-        check("Results not equal ("+num+")", joinKey, left, right, expectedResults, x1) ;
-    }
-
-    private Table joinMaterialize(JoinKey joinKey, Table left, Table right) {
-        QueryIterator qIter = join(joinKey , left , right) ;
-        return TableFactory.create(qIter) ;
-    }
-
-    public abstract QueryIterator join(JoinKey joinKey , Table left , Table right) ;
-
-    private static void check(String msg, JoinKey joinKey, Table left, Table right, Table expected, Table actual) {
-        boolean b = equalTables(expected, actual) ;
-        if ( ! b ) {
-            System.out.println("Joinkey:  "+joinKey) ;
-            System.out.println("Left:     "+left) ;
-            System.out.println("Right:    "+right) ;
-            System.out.println("Expected: "+expected) ;
-            System.out.println("Actual:   "+actual) ;
-            System.out.println() ;
-        }
-
-        assertTrue(msg, b) ;
-    }
-
-    private static boolean equalTables(Table table1, Table table2) {
-        ResultSet rs1 =  ResultSetFactory.create(table1.iterator(null), table1.getVarNames()) ;
-        ResultSet rs2 =  ResultSetFactory.create(table2.iterator(null), table2.getVarNames()) ;
-        return ResultSetCompare.equalsByTerm(rs1, rs2) ;
-    }
-
-}
-
-

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/join/TS_Join.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TS_Join.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TS_Join.java
deleted file mode 100644
index 0a4c2a6..0000000
--- a/jena-arq/src/test/java/org/apache/jena/sparql/join/TS_Join.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.join;
-
-import org.junit.runner.RunWith ;
-import org.junit.runners.Suite ;
-import org.junit.runners.Suite.SuiteClasses ;
-
-@RunWith(Suite.class)
-@SuiteClasses( {
-    TestJoinSimple.class
-    , TestJoinNestedLoopSimple.class    // Real simple materializing version.
-    , TestJoinNestedLoop.class
-    , TestHashJoin.class
-})
-
-public class TS_Join { }
-

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestHashJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestHashJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestHashJoin.java
deleted file mode 100644
index 8358bc3..0000000
--- a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestHashJoin.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.join;
-
-import java.util.Iterator ;
-
-import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-import org.apache.jena.sparql.engine.binding.Binding ;
-import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
-import org.apache.jena.sparql.engine.join.Join ;
-import org.apache.jena.sparql.engine.join.JoinKey ;
-
-public class TestHashJoin extends AbstractTestJoin {
-
-    @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
-        Iterator<Binding> data = Join.hashJoin(joinKey, left.iterator(null), right.iterator(null), null) ;
-        QueryIterator qIter = new QueryIterPlainWrapper(data) ;
-        return qIter ;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoop.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoop.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoop.java
deleted file mode 100644
index 4123c1c..0000000
--- a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoop.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.join;
-
-import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-import org.apache.jena.sparql.engine.join.Join ;
-import org.apache.jena.sparql.engine.join.JoinKey ;
-
-public class TestJoinNestedLoop extends AbstractTestJoin {
-
-    @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
-        return Join.innerLoopJoin(left.iterator(null), right.iterator(null), null) ;
-    }
-
-}
-

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoopSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoopSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoopSimple.java
deleted file mode 100644
index ff56d13..0000000
--- a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinNestedLoopSimple.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.join;
-
-import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-import org.apache.jena.sparql.engine.join.Join ;
-import org.apache.jena.sparql.engine.join.JoinKey ;
-
-public class TestJoinNestedLoopSimple extends AbstractTestJoin {
-
-    @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
-        return Join.innerLoopJoinBasic(left.iterator(null), right.iterator(null), null) ;
-    }
-
-}
-

http://git-wip-us.apache.org/repos/asf/jena/blob/d4da1a5a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinSimple.java b/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinSimple.java
deleted file mode 100644
index b7135cf..0000000
--- a/jena-arq/src/test/java/org/apache/jena/sparql/join/TestJoinSimple.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.join;
-
-import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-import org.apache.jena.sparql.engine.TableJoin ;
-import org.apache.jena.sparql.engine.join.JoinKey ;
-
-public class TestJoinSimple extends AbstractTestJoin {
-
-    @Override
-    public QueryIterator join(JoinKey joinKey, Table left, Table right) {
-        return TableJoin.join(left.iterator(null), right, null, null) ;
-    }
-
-}
-


[14/20] jena git commit: JENA-1023: Use "Join" for OpJoin and OpLeftJoin.

Posted by an...@apache.org.
JENA-1023: Use "Join" for OpJoin and OpLeftJoin.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/8f3df5a5
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/8f3df5a5
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/8f3df5a5

Branch: refs/heads/master
Commit: 8f3df5a58c77bda3b424acf3d3060de05251135f
Parents: 8a25b21
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 17:42:49 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:09:17 2015 +0100

----------------------------------------------------------------------
 .../java/org/apache/jena/sparql/engine/main/OpExecutor.java   | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/8f3df5a5/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java
index 709f735..801a48e 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/OpExecutor.java
@@ -39,6 +39,7 @@ import org.apache.jena.sparql.engine.ExecutionContext ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.binding.Binding ;
 import org.apache.jena.sparql.engine.iterator.* ;
+import org.apache.jena.sparql.engine.join.Join ;
 import org.apache.jena.sparql.engine.main.iterator.* ;
 import org.apache.jena.sparql.expr.Expr ;
 import org.apache.jena.sparql.expr.ExprList ;
@@ -218,14 +219,14 @@ public class OpExecutor
         }
         QueryIterator left = exec(opJoin.getLeft(), input) ;
         QueryIterator right = exec(opJoin.getRight(), root()) ;
-        QueryIterator qIter = new QueryIterJoin(left, right, execCxt) ;
+        // Join key.
+        QueryIterator qIter = Join.join(left, right, execCxt) ;
         return qIter ;
     }
 
     // Pass iterator from one step directly into the next.
     protected QueryIterator execute(OpSequence opSequence, QueryIterator input) {
         QueryIterator qIter = input ;
-
         for (Iterator<Op> iter = opSequence.iterator(); iter.hasNext();) {
             Op sub = iter.next() ;
             qIter = exec(sub, qIter) ;
@@ -236,7 +237,7 @@ public class OpExecutor
     protected QueryIterator execute(OpLeftJoin opLeftJoin, QueryIterator input) {
         QueryIterator left = exec(opLeftJoin.getLeft(), input) ;
         QueryIterator right = exec(opLeftJoin.getRight(), root()) ;
-        QueryIterator qIter = new QueryIterLeftJoin(left, right, opLeftJoin.getExprs(), execCxt) ;
+        QueryIterator qIter = Join.leftJoin(left, right, opLeftJoin.getExprs(), execCxt) ;
         return qIter ;
     }
 


[16/20] jena git commit: Fix comment.

Posted by an...@apache.org.
Fix comment.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/2318191b
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/2318191b
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/2318191b

Branch: refs/heads/master
Commit: 2318191b702af334284a3f11dc696a1e5388329d
Parents: 56ea4f9
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 17:40:48 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:09:17 2015 +0100

----------------------------------------------------------------------
 .../java/org/apache/jena/sparql/engine/iterator/QueryIter2.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/2318191b/jena-arq/src/main/java/org/apache/jena/sparql/engine/iterator/QueryIter2.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/iterator/QueryIter2.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/iterator/QueryIter2.java
index d017573..ba5118e 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/iterator/QueryIter2.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/iterator/QueryIter2.java
@@ -64,7 +64,7 @@ public abstract class QueryIter2 extends QueryIter
     /** Cancellation of the query execution is happening */
     protected abstract void requestSubCancel() ;
     
-    /** Pass on the close method - no need to close the left or right QueryIterators passed to the QueryIter1 constructor */
+    /** Pass on the close method - no need to close the left or right QueryIterators passed to the QueryIter2 constructor */
     protected abstract void closeSubIterator() ;
     
     // Do better


[06/20] jena git commit: Rename PLAIN as INNER.

Posted by an...@apache.org.
Rename PLAIN as INNER.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/07217a37
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/07217a37
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/07217a37

Branch: refs/heads/master
Commit: 07217a37a892b0bc68c107a2e4c87f65e984ecb5
Parents: 728542a
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 14:17:24 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:12 2015 +0100

----------------------------------------------------------------------
 .../main/java/org/apache/jena/sparql/algebra/JoinType.java  | 9 ++++++++-
 .../org/apache/jena/sparql/engine/ref/EvaluatorSimple.java  | 2 +-
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/07217a37/jena-arq/src/main/java/org/apache/jena/sparql/algebra/JoinType.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/algebra/JoinType.java b/jena-arq/src/main/java/org/apache/jena/sparql/algebra/JoinType.java
index 0dc0b5b..e0bd489 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/algebra/JoinType.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/algebra/JoinType.java
@@ -18,4 +18,11 @@
 
 package org.apache.jena.sparql.algebra;
 
-public enum JoinType { PLAIN, LEFT /*, RIGHT , /*FULL* /OUTER*/}
+public enum JoinType {
+    INNER /* Normal, plain equijoin*/ 
+    , LEFT 
+    //, RIGHT
+    //, OUTER   // Full outer join
+    //, ANTI    // Anti-join: rows in LHS where there is no matching tuple in RHS 
+    //, SEMI    // All tuples in LHS where there is a matching tuple in RHS
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/07217a37/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java
index 0f22aed..fb57584 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java
@@ -270,7 +270,7 @@ public class EvaluatorSimple implements Evaluator
     private Table joinWorker(Table tableLeft, Table tableRight, boolean leftJoin, ExprList conditions)
     {
         QueryIterator left = tableLeft.iterator(execCxt) ;
-        JoinType joinType = (leftJoin? JoinType.LEFT : JoinType.PLAIN ) ;
+        JoinType joinType = (leftJoin? JoinType.LEFT : JoinType.INNER ) ;
         QueryIterator qIter = TableJoin.joinWorker(left, tableRight, joinType, conditions, execCxt) ;
         tableLeft.close() ;
         tableRight.close() ;


[02/20] jena git commit: JENA-1023: Move simple join code into ref query engine.

Posted by an...@apache.org.
JENA-1023: Move simple join code into ref query engine.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/f165a534
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/f165a534
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/f165a534

Branch: refs/heads/master
Commit: f165a534d98b51da726f60bb2627962326e22bae
Parents: 9f158b6
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 10:21:49 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:07:41 2015 +0100

----------------------------------------------------------------------
 .../apache/jena/sparql/engine/TableJoin.java    | 104 ------------------
 .../apache/jena/sparql/engine/join/Join.java    |  74 +++++++++++++
 .../engine/main/iterator/QueryIterJoinBase.java |  73 ++++++-------
 .../jena/sparql/engine/ref/EvaluatorSimple.java |   1 -
 .../jena/sparql/engine/ref/TableJoin.java       | 106 +++++++++++++++++++
 5 files changed, 215 insertions(+), 143 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/f165a534/jena-arq/src/main/java/org/apache/jena/sparql/engine/TableJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/TableJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/TableJoin.java
deleted file mode 100644
index e16c505..0000000
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/TableJoin.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.jena.sparql.engine;
-
-import static org.apache.jena.sparql.algebra.JoinType.* ;
-
-import java.util.ArrayList ;
-import java.util.Iterator ;
-import java.util.List ;
-
-import org.apache.jena.sparql.algebra.Algebra ;
-import org.apache.jena.sparql.algebra.JoinType ;
-import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.algebra.table.TableUnit ;
-import org.apache.jena.sparql.engine.binding.Binding ;
-import org.apache.jena.sparql.engine.iterator.QueryIterFilterExpr ;
-import org.apache.jena.sparql.engine.iterator.QueryIterNullIterator ;
-import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
-import org.apache.jena.sparql.expr.Expr ;
-import org.apache.jena.sparql.expr.ExprList ;
-
-/** Table join - this only happens if the patterns can not be streamed. */  
-public class TableJoin
-{
-    public static QueryIterator join(QueryIterator left, Table right, ExprList condition, ExecutionContext execCxt) {
-        return joinWorker(left, right, PLAIN, condition, execCxt) ;
-    }
-    
-    public static QueryIterator leftJoin(QueryIterator left, Table right, ExprList condition, ExecutionContext execCxt) {
-        return joinWorker(left, right, LEFT, condition, execCxt) ;
-    }
-
-    public static QueryIterator joinWorker(QueryIterator left, Table right, JoinType joinType, ExprList conditions, ExecutionContext execCxt) {
-        if ( right.isEmpty() ) {
-            if ( joinType == PLAIN ) {
-                // No rows - no match
-                left.close() ;
-                return QueryIterNullIterator.create(execCxt) ;
-            }
-            else
-                return left ;
-        }
-        
-        if ( TableUnit.isTableUnit(right) ) {
-            if ( joinType == PLAIN )
-                return applyConditions(left, conditions, execCxt) ;
-            else
-                return left ;
-        }
-        return joinWorkerN(left, right, joinType, conditions, execCxt) ;
-    }
-            
-    private static QueryIterator joinWorkerN(QueryIterator left, Table right, JoinType joinType, ExprList conditions, ExecutionContext execCxt) {       
-        // We could hash the right except we don't know much about columns.
-        
-        List<Binding> out = new ArrayList<>() ;
-        for ( ; left.hasNext() ; ) {
-            Binding bindingLeft = left.next() ;
-            int count = 0 ;
-            for (Iterator<Binding> iter = right.rows() ; iter.hasNext();) {
-                Binding bindingRight = iter.next() ;
-                Binding r = Algebra.merge(bindingLeft, bindingRight) ;
-                
-                if ( r == null )
-                    continue ;
-                // This does the conditional part. Theta-join.
-                if ( conditions == null || conditions.isSatisfied(r, execCxt) ) {
-                    count ++ ;
-                    out.add(r) ;
-                }
-            }
-
-            if ( count == 0 && ( joinType == LEFT)  )
-                // Conditions on left?
-                out.add(bindingLeft) ;
-        }
-        
-        return new QueryIterPlainWrapper(out.iterator(), execCxt) ;
-    }
-    
-    private static QueryIterator applyConditions(QueryIterator qIter, ExprList conditions, ExecutionContext execCxt) {
-        if ( conditions == null )
-            return qIter ;
-        for (Expr expr : conditions)
-            qIter = new QueryIterFilterExpr(qIter, expr, execCxt) ;
-        return qIter ;
-    }
-}

http://git-wip-us.apache.org/repos/asf/jena/blob/f165a534/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
index 0a71cc1..68536bf 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
@@ -18,19 +18,93 @@
 
 package org.apache.jena.sparql.engine.join;
 
+import static org.apache.jena.sparql.algebra.JoinType.LEFT ;
+import static org.apache.jena.sparql.algebra.JoinType.PLAIN ;
+
 import java.util.List ;
 
 import org.apache.jena.atlas.iterator.Iter ;
 import org.apache.jena.atlas.lib.DS ;
 import org.apache.jena.sparql.algebra.Algebra ;
+import org.apache.jena.sparql.algebra.JoinType ;
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.algebra.TableFactory ;
 import org.apache.jena.sparql.engine.ExecutionContext ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.binding.Binding ;
 import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
+import org.apache.jena.sparql.engine.main.OpExecutor ;
+import org.apache.jena.sparql.engine.ref.TableJoin ;
+import org.apache.jena.sparql.expr.ExprList ;
 
 /** API to various join algorithms */
 public class Join {
     // See also package org.apache.jena.sparql.engine.index
+
+    /**
+     * Standard entry point to a join of two streams.
+     * This is not a substitution/index join.
+     * (See {@link OpExecutor} for streamed execution using substitution).
+     * @param left
+     * @param right
+     * @param execCxt
+     * @return QueryIterator
+     */
+    public static QueryIterator join(QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
+        return joinWorker(left, right, PLAIN, null, execCxt) ;
+    }
+   
+    /** Standard entry point to a left join of two streams.
+     * This is not a substitution/index join.
+     * (See {@link OpExecutor} for streamed execution using substitution).
+     *
+     * @param left
+     * @param right
+     * @param conditions
+     * @param execCxt
+     * @return QueryIterator
+     */
+    public static QueryIterator leftJoin(QueryIterator left, QueryIterator right, ExprList conditions, ExecutionContext execCxt) {
+        return joinWorker(left, right, LEFT, conditions, execCxt) ;
+    }
+   
+
+    /**
+     * Standard entry point to a join of two streams.
+     * This is not a substitution/index join.
+     * (See {@link OpExecutor} for streamed execution using substitution).
+     * @param left
+     * @param right
+     * @param joinType
+     * @param conditions
+     * @param execCxt
+     * @return QueryIterator
+     */
+    public static QueryIterator joinWorker(QueryIterator left, QueryIterator right, 
+                                           JoinType joinType, ExprList conditions,
+                                           ExecutionContext execCxt) {
+        // Catch easy cases.
+        // If left or right is a root, do fast.
+//        if ( right.isEmpty() ) {
+//            if ( joinType == PLAIN ) {
+//                // No rows - no match
+//                left.close() ;
+//                return QueryIterNullIterator.create(execCxt) ;
+//            }
+//            else
+//                // Left join - pass out left rows regardless of conditions.
+//                return left ;
+//        }
+//        
+//        if ( TableUnit.isTableUnit(right) )
+//            return applyConditions(left, conditions, execCxt) ;
+//        return joinWorkerN(left, right, joinType, conditions, execCxt) ;
+        // XXX TEMPORARY!
+        //throw new NotImplemented() ;
+        Table t = TableFactory.create(right) ;
+        return TableJoin.joinWorker(left, t, joinType, conditions, execCxt) ;
+    }
+    
     
     /** Evaluate a hash join.  This code materializes the left into a probe table
      * then hash joins from the right.  

http://git-wip-us.apache.org/repos/asf/jena/blob/f165a534/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java
index c346ecf..3fb00dc 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/main/iterator/QueryIterJoinBase.java
@@ -18,62 +18,59 @@
 
 package org.apache.jena.sparql.engine.main.iterator;
 
-import org.apache.jena.sparql.algebra.JoinType ;
-import org.apache.jena.sparql.algebra.Table ;
-import org.apache.jena.sparql.algebra.TableFactory ;
-import org.apache.jena.sparql.engine.ExecutionContext ;
-import org.apache.jena.sparql.engine.QueryIterator ;
-import org.apache.jena.sparql.engine.TableJoin ;
-import org.apache.jena.sparql.engine.binding.Binding ;
-import org.apache.jena.sparql.engine.iterator.QueryIter ;
-import org.apache.jena.sparql.expr.ExprList ;
+import org.apache.jena.sparql.algebra.JoinType;
+import org.apache.jena.sparql.engine.ExecutionContext;
+import org.apache.jena.sparql.engine.QueryIterator;
+import org.apache.jena.sparql.engine.binding.Binding;
+import org.apache.jena.sparql.engine.iterator.QueryIter;
+import org.apache.jena.sparql.engine.join.Join;
+import org.apache.jena.sparql.expr.ExprList;
 
-/** Join or LeftJoin by calculating both sides, then doing the join
- *  It usually better to use substitute algorithm (not this
- *  QueryIterator in other words) as that is effectively indexing
- *  from one side into the other. */ 
-public class QueryIterJoinBase extends QueryIter
-{
+/**
+ * Join or LeftJoin by calculating both sides, then doing the join It usually
+ * better to use substitute algorithm (not this QueryIterator in other words) as
+ * that is effectively indexing from one side into the other.
+ */
+public class QueryIterJoinBase extends QueryIter {
     // This should be converted to a hash or sort-merge join.
-    private final QueryIterator left ;
-    private final QueryIterator right ;
-    private final QueryIterator result ;
-    
-    protected QueryIterJoinBase(QueryIterator left, QueryIterator right, JoinType joinType, ExprList exprs, ExecutionContext execCxt)
-    {
-        super(execCxt) ;
-        this.left = left ;
-        this.right = right ;
-        this.result = calc(left, right, joinType, exprs, execCxt) ; 
-    }
+    private final QueryIterator left;
+    private final QueryIterator right;
+    private final QueryIterator result;
 
-    private static QueryIterator calc(QueryIterator left, QueryIterator right, JoinType joinType, ExprList exprs, ExecutionContext execCxt) {
-        Table tableRight = TableFactory.create(right) ;
-        return TableJoin.joinWorker(left, tableRight, joinType, exprs, execCxt) ;
+    protected QueryIterJoinBase(QueryIterator left, QueryIterator right, JoinType joinType, ExprList exprs, ExecutionContext execCxt) {
+        super(execCxt);
+        this.left = left;
+        this.right = right;
+        this.result = calc(left, right, joinType, exprs, execCxt);
+    }
 
+    private static QueryIterator calc(QueryIterator left, QueryIterator right,
+                                      JoinType joinType, ExprList exprs,
+                                      ExecutionContext execCxt) {
+        return Join.joinWorker(left, right, joinType, exprs, execCxt);
     }
-    
+
     @Override
     protected boolean hasNextBinding() {
-        return result.hasNext() ;
+        return result.hasNext();
     }
 
     @Override
     protected Binding moveToNextBinding() {
-        return result.nextBinding() ;
+        return result.nextBinding();
     }
 
     @Override
     protected void closeIterator() {
-        left.close() ;
-        right.close() ;
-        result.close() ;
+        left.close();
+        right.close();
+        result.close();
     }
 
     @Override
     protected void requestCancel() {
-        left.cancel() ;
-        right.cancel() ;
-        result.cancel() ;
+        left.cancel();
+        right.cancel();
+        result.cancel();
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/f165a534/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java
index d79fe12..0f22aed 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/EvaluatorSimple.java
@@ -35,7 +35,6 @@ import org.apache.jena.sparql.core.VarExprList ;
 import org.apache.jena.sparql.engine.ExecutionContext ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.ResultSetStream ;
-import org.apache.jena.sparql.engine.TableJoin ;
 import org.apache.jena.sparql.engine.binding.Binding ;
 import org.apache.jena.sparql.engine.iterator.* ;
 import org.apache.jena.sparql.engine.main.QC ;

http://git-wip-us.apache.org/repos/asf/jena/blob/f165a534/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java
new file mode 100644
index 0000000..483764e
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java
@@ -0,0 +1,106 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.ref;
+
+import static org.apache.jena.sparql.algebra.JoinType.* ;
+
+import java.util.ArrayList ;
+import java.util.Iterator ;
+import java.util.List ;
+
+import org.apache.jena.sparql.algebra.Algebra ;
+import org.apache.jena.sparql.algebra.JoinType ;
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.algebra.table.TableUnit ;
+import org.apache.jena.sparql.engine.ExecutionContext ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.binding.Binding ;
+import org.apache.jena.sparql.engine.iterator.QueryIterFilterExpr ;
+import org.apache.jena.sparql.engine.iterator.QueryIterNullIterator ;
+import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
+import org.apache.jena.sparql.expr.Expr ;
+import org.apache.jena.sparql.expr.ExprList ;
+
+/** Table join - this only happens if the patterns can not be streamed.
+ * This code is simple! 
+ * Primarily for use in testing and also the reference query engine which is designed for simplicity, not performance.
+ */  
+public class TableJoin
+{
+    public static QueryIterator join(QueryIterator left, Table right, ExprList condition, ExecutionContext execCxt) {
+        return joinWorker(left, right, PLAIN, condition, execCxt) ;
+    }
+    
+    public static QueryIterator leftJoin(QueryIterator left, Table right, ExprList condition, ExecutionContext execCxt) {
+        return joinWorker(left, right, LEFT, condition, execCxt) ;
+    }
+
+    public static QueryIterator joinWorker(QueryIterator left, Table right, JoinType joinType, ExprList conditions, ExecutionContext execCxt) {
+        if ( right.isEmpty() ) {
+            if ( joinType == PLAIN ) {
+                // No rows - no match
+                left.close() ;
+                return QueryIterNullIterator.create(execCxt) ;
+            }
+            else
+                // Left join - pass out left rows regardless of conditions.
+                return left ;
+        }
+        
+        if ( TableUnit.isTableUnit(right) )
+            return applyConditions(left, conditions, execCxt) ;
+        return joinWorkerN(left, right, joinType, conditions, execCxt) ;
+    }
+            
+    private static QueryIterator joinWorkerN(QueryIterator left, Table right, JoinType joinType, ExprList conditions, ExecutionContext execCxt) {       
+        // We could hash the right except we don't know much about columns.
+        
+        List<Binding> out = new ArrayList<>() ;
+        for ( ; left.hasNext() ; ) {
+            Binding bindingLeft = left.next() ;
+            int count = 0 ;
+            for (Iterator<Binding> iter = right.rows() ; iter.hasNext();) {
+                Binding bindingRight = iter.next() ;
+                Binding r = Algebra.merge(bindingLeft, bindingRight) ;
+                
+                if ( r == null )
+                    continue ;
+                // This does the conditional part. Theta-join.
+                if ( conditions == null || conditions.isSatisfied(r, execCxt) ) {
+                    count ++ ;
+                    out.add(r) ;
+                }
+            }
+
+            if ( count == 0 && ( joinType == LEFT)  )
+                // Conditions on left?
+                out.add(bindingLeft) ;
+        }
+        
+        return new QueryIterPlainWrapper(out.iterator(), execCxt) ;
+    }
+    
+    private static QueryIterator applyConditions(QueryIterator qIter, ExprList conditions, ExecutionContext execCxt) {
+        if ( conditions == null )
+            return qIter ;
+        for (Expr expr : conditions)
+            qIter = new QueryIterFilterExpr(qIter, expr, execCxt) ;
+        return qIter ;
+    }
+}


[05/20] jena git commit: Correct naming. Expand javadoc.

Posted by an...@apache.org.
Correct naming. Expand javadoc.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/24e4c1ef
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/24e4c1ef
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/24e4c1ef

Branch: refs/heads/master
Commit: 24e4c1ef3f66d5ffc23666988881f870a94d18fe
Parents: 07217a3
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 14:17:52 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:12 2015 +0100

----------------------------------------------------------------------
 .../apache/jena/sparql/engine/join/Join.java    | 89 ++++++++++++++++----
 1 file changed, 74 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/24e4c1ef/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
index 68536bf..4d4523f 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/join/Join.java
@@ -19,7 +19,7 @@
 package org.apache.jena.sparql.engine.join;
 
 import static org.apache.jena.sparql.algebra.JoinType.LEFT ;
-import static org.apache.jena.sparql.algebra.JoinType.PLAIN ;
+import static org.apache.jena.sparql.algebra.JoinType.INNER ;
 
 import java.util.List ;
 
@@ -32,15 +32,22 @@ import org.apache.jena.sparql.algebra.TableFactory ;
 import org.apache.jena.sparql.engine.ExecutionContext ;
 import org.apache.jena.sparql.engine.QueryIterator ;
 import org.apache.jena.sparql.engine.binding.Binding ;
+import org.apache.jena.sparql.engine.iterator.QueryIterFilterExpr ;
 import org.apache.jena.sparql.engine.iterator.QueryIterPlainWrapper ;
 import org.apache.jena.sparql.engine.main.OpExecutor ;
 import org.apache.jena.sparql.engine.ref.TableJoin ;
+import org.apache.jena.sparql.expr.Expr ;
 import org.apache.jena.sparql.expr.ExprList ;
 
 /** API to various join algorithms */
 public class Join {
     // See also package org.apache.jena.sparql.engine.index
+    // The anti-join code for MINUS
 
+    // *******************
+    // USES OLD TableJoin CODE AT THE MOMENT
+    // *******************
+    
     /**
      * Standard entry point to a join of two streams.
      * This is not a substitution/index join.
@@ -51,7 +58,7 @@ public class Join {
      * @return QueryIterator
      */
     public static QueryIterator join(QueryIterator left, QueryIterator right, ExecutionContext execCxt) {
-        return joinWorker(left, right, PLAIN, null, execCxt) ;
+        return joinWorker(left, right, INNER, null, execCxt) ;
     }
    
     /** Standard entry point to a left join of two streams.
@@ -106,8 +113,32 @@ public class Join {
     }
     
     
+    /** Inner loop join.
+     *  Cancellable.
+     * @param left      Left hand side
+     * @param right     Right hand side
+     * @param cxt       ExecutionContext
+     * @return          QueryIterator
+     */ 
+    public static QueryIterator nestedLoopJoin(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
+        return new QueryIterNestedLoopJoin(left, right, cxt) ;
+    }
+
+    /** Inner loop join.
+     *  Cancellable.
+     * @param left      Left hand side
+     * @param right     Right hand side
+     * @param cxt       ExecutionContext
+     * @return          QueryIterator
+     */ 
+    public static QueryIterator nestedLoopLeftJoin(QueryIterator left, QueryIterator right, ExprList conditions, ExecutionContext cxt) {
+        return new QueryIterNestedLoopLeftJoin(left, right, conditions, cxt) ;
+    }
+
+
     /** Evaluate a hash join.  This code materializes the left into a probe table
      * then hash joins from the right.  
+     *  No cancelation, no stats.
      * 
      * @param joinKey   The key for the probe table.
      * @param left      Left hand side
@@ -119,10 +150,14 @@ public class Join {
         return new QueryIterHashJoin(joinKey, left, right, cxt) ;
     }
 
-    /** Very simple, materializing version - useful for debugging. 
-     * Does <b>not</b> scale. 
+    /** Very simple, materializing version - useful for debugging.
+     *  Builds output early. Materializes left, streams right.
+     *  Does <b>not</b> scale. 
+     *  No cancelation, no stats.
+     * 
+     * @see #nestedLoopJoin
      */
-    public static QueryIterator innerLoopJoinBasic(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
+    public static QueryIterator nestedLoopJoinBasic(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
         List<Binding> leftRows = Iter.toList(left) ;
         List<Binding> output = DS.list() ;
         for ( ; right.hasNext() ; ) {
@@ -136,16 +171,40 @@ public class Join {
         return new QueryIterPlainWrapper(output.iterator(), cxt) ;
     }
 
-    /** Inner loops join.  This code materializes the left ino the inner loop tabole and
-     *  then streams on the right.
-     *  
-     * @param left      Left hand side
-     * @param right     Right hand side
-     * @param cxt       ExecutionContext
-     * @return          QueryIterator
-     */ 
-    public static QueryIterator innerLoopJoin(QueryIterator left, QueryIterator right, ExecutionContext cxt) {
-        return new QueryIterNestedLoopJoin(left, right, cxt) ;
+    /** Very simple, materializing version for leftjoin - useful for debugging.
+     *  Builds output early. Materializes right, streams left.
+     *  Does <b>not</b> scale. 
+     */
+    public static QueryIterator nestedLoopLeftJoinBasic(QueryIterator left, QueryIterator right, ExprList condition, ExecutionContext cxt) {
+        // Stream from left, materialize right.
+        List<Binding> rightRows = Iter.toList(right) ;
+        List<Binding> output = DS.list() ;
+        long count = 0 ;
+        for ( ; left.hasNext() ; ) {
+            Binding row1 = left.next() ;
+            boolean match = false ;
+            for ( Binding row2 : rightRows ) {
+                Binding r = Algebra.merge(row1, row2) ;
+                if ( r != null ) {
+                    output.add(r) ;
+                    match = true ;
+                }
+            }
+            if ( ! match )
+                output.add(row1) ;
+        }
+        QueryIterator qIter = new QueryIterPlainWrapper(output.iterator(), cxt) ;
+        qIter = applyConditions(qIter, condition, cxt) ;
+        return qIter ;
+    }
+
+    // apply conditions.
+    private static QueryIterator applyConditions(QueryIterator qIter, ExprList conditions, ExecutionContext execCxt) {
+        if ( conditions == null )
+            return qIter ;
+        for (Expr expr : conditions)
+            qIter = new QueryIterFilterExpr(qIter, expr, execCxt) ;
+        return qIter ;
     }
 }
 


[09/20] jena git commit: Drop into isomorphicsm testing if not simply equivalent.

Posted by an...@apache.org.
Drop into isomorphicsm testing if not simply equivalent.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/13294cb8
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/13294cb8
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/13294cb8

Branch: refs/heads/master
Commit: 13294cb82da7ac6c3c74b750897fb0d62680ce72
Parents: d4da1a5
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 11:45:43 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:12 2015 +0100

----------------------------------------------------------------------
 .../java/org/apache/jena/sparql/resultset/ResultSetCompare.java   | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/13294cb8/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultSetCompare.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultSetCompare.java b/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultSetCompare.java
index 34e793a..eb3c329 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultSetCompare.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/resultset/ResultSetCompare.java
@@ -133,11 +133,12 @@ public class ResultSetCompare
         ResultSetRewindable rs1a = ResultSetFactory.makeRewindable(rs1) ;
         ResultSetRewindable rs2a = ResultSetFactory.makeRewindable(rs2) ;
         
+        // Aligned rows
         if ( equivalent(convert(rs1a), convert(rs2a), new BNodeIso(NodeUtils.sameTerm)) )
             return true ;
         rs1a.reset() ;    
         rs2a.reset() ;
-        return isomorphic(rs1, rs2) ;
+        return isomorphic(rs1a, rs2a) ;
     }
 
     


[04/20] jena git commit: Rename to make clearer

Posted by an...@apache.org.
Rename to make clearer


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/0b060c80
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/0b060c80
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/0b060c80

Branch: refs/heads/master
Commit: 0b060c80c60cb428e993d162d5bb74ebeff6ac5e
Parents: f165a53
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 10:22:29 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:11 2015 +0100

----------------------------------------------------------------------
 .../jena/sparql/engine/ref/TestTableJoin.java   | 120 +++++++++++++++++++
 1 file changed, 120 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/0b060c80/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java b/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java
new file mode 100644
index 0000000..b52d532
--- /dev/null
+++ b/jena-arq/src/test/java/org/apache/jena/sparql/engine/ref/TestTableJoin.java
@@ -0,0 +1,120 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.jena.sparql.engine.ref;
+
+import java.util.HashSet ;
+import java.util.Set ;
+
+import org.apache.jena.atlas.junit.BaseTest ;
+import org.apache.jena.query.ARQ ;
+import org.apache.jena.query.ResultSet ;
+import org.apache.jena.query.ResultSetFactory ;
+import org.apache.jena.sparql.algebra.Table ;
+import org.apache.jena.sparql.algebra.table.TableEmpty ;
+import org.apache.jena.sparql.algebra.table.TableN ;
+import org.apache.jena.sparql.algebra.table.TableUnit ;
+import org.apache.jena.sparql.engine.ExecutionContext ;
+import org.apache.jena.sparql.engine.QueryIterator ;
+import org.apache.jena.sparql.engine.ref.TableJoin ;
+import org.apache.jena.sparql.expr.ExprList ;
+import org.apache.jena.sparql.resultset.ResultSetCompare ;
+import org.apache.jena.sparql.sse.SSE ;
+import org.junit.Test ;
+
+/** Tests on tables and the simple nest loop join (TableJoin) used by the reference query engine */ 
+public class TestTableJoin extends BaseTest
+{
+    Table unit = new TableUnit() ; 
+    Table zero = new TableEmpty() ;
+    Table zeroData = SSE.parseTable("(table)") ;
+    Table unitData = SSE.parseTable("(table (row))") ;
+    Table data1 = SSE.parseTable("(table (row (?a 1) (?b 2)))") ;
+    Table data2 = SSE.parseTable("(table (row (?a 1) (?c 3)) (row (?a 9) (?c 5))  )") ;
+    Table data3 = SSE.parseTable("(table (row (?a 1) (?c 3)) (row (?a 1) (?c 4)) (row (?a 9) (?c 5))  )") ;
+    
+    Table data1J2 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) )") ;
+    Table data1LJ2 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) (row (?a 9) (?c 5)) )") ;
+    Table data2LJ1 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) )") ;
+    
+    Table data1J3 = SSE.parseTable("(table (row (?a 1) (?b 2) (?c 3)) (row (?a 1) (?b 2) (?c 4)) )") ;
+    
+    @Test public void table_01() { test(unit, zero, true, null, zero) ; }
+    @Test public void table_02() { test(zero, unit, true, null, zero) ; }
+    @Test public void table_03() { test(unit, zero, false, null, unit) ; }
+    @Test public void table_04() { test(zero, unit, false, null, zero) ; }
+    // Same again but with non-special tables.
+    @Test public void table_05() { test(unitData, zeroData, true, null, zeroData) ; }
+    @Test public void table_06() { test(zeroData, unitData, true, null, zeroData) ; }
+    @Test public void table_07() { test(unitData, zeroData, false, null, unitData) ; }
+    @Test public void table_08() { test(zeroData, unitData, false, null, zeroData) ; }
+
+    @Test public void table_10() { test(data1, zero, true, null, zero) ; }
+    @Test public void table_11() { test(zero, data1, true, null, zero) ; }
+    @Test public void table_12() { test(data1, zero, false, null, data1) ; }
+    @Test public void table_13() { test(zero, data1, false, null, zero) ; }
+    
+    @Test public void table_14() { test(data1, zeroData, true, null, zeroData) ; }
+    @Test public void table_15() { test(zeroData, data1, true, null, zeroData) ; }
+    @Test public void table_16() { test(data1, zeroData, false, null, data1) ; }
+    @Test public void table_17() { test(zeroData, data1, false, null, zeroData) ; }
+
+    @Test public void table_18() { test(data2, unitData, true, null, data2) ; }
+    @Test public void table_19() { test(unitData, data2, true, null, data2) ; }
+    
+    @Test public void table_20() { test(data1, data2, true, null, data1J2) ; }
+    @Test public void table_21() { test(data2, data1, true, null, data1J2) ; }
+    @Test public void table_22() { test(data1, data2, false, null, data1LJ2) ; }
+    @Test public void table_23() { test(data2, data1, false, null, data2LJ1) ; }
+    
+    @Test public void table_24() { test(data1, data3, true, null, data1J3) ; }
+    @Test public void table_25() { test(data3, data1, true, null, data1J3) ; }
+
+    private void test(Table left, Table right, boolean normalJoin, ExprList exprs, Table expected) {
+        ExecutionContext execCxt = new ExecutionContext(ARQ.getContext(), null, null, null) ;
+        QueryIterator leftIter = left.iterator(execCxt) ;
+        QueryIterator qIter = normalJoin 
+            ? TableJoin.join(leftIter, right, exprs, execCxt)
+            : TableJoin.leftJoin(leftIter, right, exprs, execCxt) ;
+        
+            // Order issues
+            
+        Set<String> vars1 = new HashSet<>() ;
+        vars1.addAll(left.getVarNames()) ;
+        vars1.addAll(right.getVarNames()) ;
+        
+        TableN results = new TableN(qIter) ;
+        boolean b = TableCompare.equalsByTerm(expected, results) ;
+        if ( !b ) {
+            System.out.println("** Expected") ;
+            System.out.println(expected) ;
+            System.out.println("** Actual") ;
+            System.out.println(results) ;
+        }
+        assertTrue(b) ;
+    }
+    
+    static class TableCompare {
+        public static boolean equalsByTerm(Table table1, Table table2) {
+            ResultSet rs1 = ResultSetFactory.create(table1.iterator(null), table1.getVarNames()) ;
+            ResultSet rs2 = ResultSetFactory.create(table2.iterator(null), table2.getVarNames()) ;
+            return ResultSetCompare.equalsByTerm(rs1, rs2) ;
+        }
+    }
+        
+}


[12/20] jena git commit: Rename in JoinType.

Posted by an...@apache.org.
Rename in JoinType.


Project: http://git-wip-us.apache.org/repos/asf/jena/repo
Commit: http://git-wip-us.apache.org/repos/asf/jena/commit/6da3c8d9
Tree: http://git-wip-us.apache.org/repos/asf/jena/tree/6da3c8d9
Diff: http://git-wip-us.apache.org/repos/asf/jena/diff/6da3c8d9

Branch: refs/heads/master
Commit: 6da3c8d98cd9b8302cf760098e8a581a9e6ac4a7
Parents: eccf873
Author: Andy Seaborne <an...@apache.org>
Authored: Tue Sep 8 14:19:12 2015 +0100
Committer: Andy Seaborne <an...@apache.org>
Committed: Tue Sep 8 18:08:55 2015 +0100

----------------------------------------------------------------------
 .../main/java/org/apache/jena/sparql/engine/ref/TableJoin.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/6da3c8d9/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java b/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java
index 483764e..acdb5cc 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/engine/ref/TableJoin.java
@@ -44,7 +44,7 @@ import org.apache.jena.sparql.expr.ExprList ;
 public class TableJoin
 {
     public static QueryIterator join(QueryIterator left, Table right, ExprList condition, ExecutionContext execCxt) {
-        return joinWorker(left, right, PLAIN, condition, execCxt) ;
+        return joinWorker(left, right, INNER, condition, execCxt) ;
     }
     
     public static QueryIterator leftJoin(QueryIterator left, Table right, ExprList condition, ExecutionContext execCxt) {
@@ -53,7 +53,7 @@ public class TableJoin
 
     public static QueryIterator joinWorker(QueryIterator left, Table right, JoinType joinType, ExprList conditions, ExecutionContext execCxt) {
         if ( right.isEmpty() ) {
-            if ( joinType == PLAIN ) {
+            if ( joinType == INNER ) {
                 // No rows - no match
                 left.close() ;
                 return QueryIterNullIterator.create(execCxt) ;