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 2016/01/30 22:41:41 UTC

[2/9] jena git commit: Factoring tuple ordering into TupleMap

Factoring tuple ordering into TupleMap


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

Branch: refs/heads/master
Commit: f0666056f0d27e7f8666c8819530b1bc922db1d7
Parents: ca5f09d
Author: ajs6f <aj...@virginia.edu>
Authored: Wed Jan 6 11:15:35 2016 -0500
Committer: ajs6f <aj...@virginia.edu>
Committed: Sun Jan 17 15:40:26 2016 -0500

----------------------------------------------------------------------
 .../jena/sparql/core/mem/OrderedTupleTable.java | 103 +++++++++++++
 .../jena/sparql/core/mem/PMapQuadTable.java     | 123 +++++++++------
 .../jena/sparql/core/mem/PMapTripleTable.java   | 117 ++++++++------
 .../jena/sparql/core/mem/PMapTupleTable.java    |  17 ++-
 .../apache/jena/sparql/core/mem/QuadTable.java  |  68 ++++++---
 .../jena/sparql/core/mem/QuadTableForm.java     | 152 ++-----------------
 .../jena/sparql/core/mem/TripleTable.java       |  55 +++++--
 .../jena/sparql/core/mem/TripleTableForm.java   | 101 ++----------
 .../jena/atlas/lib/tuple/QuadConsumer.java      |  42 +++++
 .../jena/atlas/lib/tuple/QuadFunction.java      |  45 ++++++
 .../jena/atlas/lib/tuple/TriConsumer.java       |  42 +++++
 .../jena/atlas/lib/tuple/TriFunction.java       |  42 +++++
 .../apache/jena/atlas/lib/tuple/TupleMap.java   |   5 +-
 13 files changed, 541 insertions(+), 371 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/OrderedTupleTable.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/OrderedTupleTable.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/OrderedTupleTable.java
new file mode 100644
index 0000000..21f2852
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/OrderedTupleTable.java
@@ -0,0 +1,103 @@
+/*
+ * 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.core.mem;
+
+import static org.apache.jena.sparql.core.mem.QuadTable.accept;
+import static org.apache.jena.sparql.core.mem.QuadTable.apply;
+import static org.apache.jena.sparql.core.mem.TripleTable.accept;
+import static org.apache.jena.sparql.core.mem.TripleTable.apply;
+
+import java.util.function.Consumer;
+
+import org.apache.jena.atlas.lib.tuple.TupleMap;
+import org.apache.jena.atlas.lib.tuple.QuadConsumer.Consumer4;
+import org.apache.jena.atlas.lib.tuple.QuadFunction.QuadOperator;
+import org.apache.jena.atlas.lib.tuple.TriConsumer.Consumer3;
+import org.apache.jena.atlas.lib.tuple.TriFunction.TriOperator;
+import org.apache.jena.graph.Node;
+import org.apache.jena.graph.Triple;
+import org.apache.jena.sparql.core.Quad;
+
+/**
+ * A table of {@code TupleType} tuples that uses an internal order recorded via {@link TupleMap}.
+ * 
+ * @param <TupleType> the tuple type in which an instance of this class works, typically {@link Triple} or {@link Quad}
+ * @param <ConsumerType> a consumer type that can accept the elements of a {@link TupleType}, typically
+ *            {@link Consumer3} or {@link Consumer4}
+ */
+public abstract class OrderedTupleTable<TupleType, ConsumerType> implements TupleTable<TupleType> {
+
+    /**
+     * The order in which elements are held in this table, e.g. SPO or OSGP.
+     */
+    protected final TupleMap order;
+
+    /**
+     * The reverse of {@link #order}.
+     */
+    protected final TupleMap reverse;
+
+    /**
+     * @param order the order of elements in this table
+     */
+    public OrderedTupleTable(final TupleMap order) {
+        this.order = order;
+        this.reverse = order.reverse();
+    }
+
+    /**
+     * @return a consumer that adds a tuple in the form of the elements in a {@code TupleType} to this table
+     */
+    protected abstract ConsumerType add();
+
+    /**
+     * @return a consumer that removes a tuple in the form of the elements in a {@code TupleType} from this table
+     */
+    protected abstract ConsumerType delete();
+
+    protected Consumer<Quad> map(final Consumer4<Node> consumer) {
+        return q -> {
+            final Node g = q.getGraph();
+            final Node s = q.getSubject();
+            final Node p = q.getPredicate();
+            final Node o = q.getObject();
+            accept(order, g, s, p, o, consumer);
+        };
+    }
+
+    protected <X> QuadOperator<Node, X> map(final QuadOperator<Node, X> f) {
+        return (g, s, p, o) -> apply(order, g, s, p, o, f);
+    }
+
+    protected Quad unmap(final Node first, final Node second, final Node third, final Node fourth) {
+        return apply(reverse, first, second, third, fourth, Quad::new);
+    }
+
+    protected Consumer<Triple> map(final Consumer3<Node> consumer) {
+        return t -> {
+            final Node s = t.getSubject();
+            final Node p = t.getPredicate();
+            final Node o = t.getObject();
+            accept(order, s, p, o, consumer);
+        };
+    }
+
+    protected <T, X> TriOperator<T, X> map(final TriOperator<T, X> f) {
+        return (s, p, o) -> apply(order, s, p, o, f);
+    }
+
+    protected Triple unmap(final Node first, final Node second, final Node third) {
+        return apply(reverse, first, second, third, Triple::new);
+    }
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapQuadTable.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapQuadTable.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapQuadTable.java
index 9606c6f..cd82f48 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapQuadTable.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapQuadTable.java
@@ -17,7 +17,6 @@
  */
 
 package org.apache.jena.sparql.core.mem;
-
 import static java.util.stream.Stream.empty;
 import static java.util.stream.Stream.of;
 import static org.slf4j.LoggerFactory.getLogger;
@@ -26,6 +25,9 @@ import java.util.stream.Stream;
 
 import org.apache.jena.atlas.lib.persistent.PMap;
 import org.apache.jena.atlas.lib.persistent.PersistentSet;
+import org.apache.jena.atlas.lib.tuple.QuadConsumer.Consumer4;
+import org.apache.jena.atlas.lib.tuple.QuadFunction.QuadOperator;
+import org.apache.jena.atlas.lib.tuple.TupleMap;
 import org.apache.jena.graph.Node;
 import org.apache.jena.sparql.core.Quad;
 import org.apache.jena.sparql.core.mem.FourTupleMap.ThreeTupleMap;
@@ -37,13 +39,29 @@ import org.slf4j.Logger;
  * use.
  *
  */
-public abstract class PMapQuadTable extends PMapTupleTable<FourTupleMap, Quad>implements QuadTable {
+public class PMapQuadTable extends PMapTupleTable<FourTupleMap, Quad, Consumer4<Node>>implements QuadTable {
+
+
+    private static class QuadTupleMap extends TupleMap {
+
+        private QuadTupleMap(final String mappedOrder) {
+            super(mappedOrder, compileMapping("GSPO", mappedOrder));
+        }
+    }
+
+    /**
+     * @param order a name and order for this table
+     */
+    public PMapQuadTable(final String order) {
+        this(order, new QuadTupleMap(order));
+    }
 
     /**
      * @param tableName a name for this table
+     * @param order the order of elements in this table
      */
-    public PMapQuadTable(final String tableName) {
-        super(tableName);
+    public PMapQuadTable(final String tableName, final TupleMap order) {
+        super(tableName, order);
     }
 
     private static final Logger log = getLogger(PMapQuadTable.class);
@@ -57,19 +75,23 @@ public abstract class PMapQuadTable extends PMapTupleTable<FourTupleMap, Quad>im
     protected FourTupleMap initial() {
         return new FourTupleMap();
     }
+    
+    
+    @Override
+    public void add(final Quad q) {
+        map(add()).accept(q);
+    }
 
-    /**
-     * Constructs a {@link Quad} from the nodes given, using the appropriate order for this table. E.g. a OPSG table
-     * should return a {@code Quad} using ({@code fourth}, {@code third}, {@code second}, {@code first}).
-     *
-     * @param first
-     * @param second
-     * @param third
-     * @param fourth
-     * @return a {@code Quad}
-     */
-    protected abstract Quad quad(final Node first, final Node second, final Node third, final Node fourth);
+    @Override
+    public void delete(final Quad q) {
+        map(delete()).accept(q);
+    }
 
+    @Override
+    public Stream<Quad> find(Node g, Node s, Node p, Node o) {
+        return map(find).apply(g, s, p, o);
+    }
+    
     /**
      * We descend through the nested {@link PMap}s building up {@link Stream}s of partial tuples from which we develop a
      * {@link Stream} of full tuples which is our result. Use {@link Node#ANY} or <code>null</code> for a wildcard.
@@ -81,7 +103,7 @@ public abstract class PMapQuadTable extends PMapTupleTable<FourTupleMap, Quad>im
      * @return a <code>Stream</code> of tuples matching the pattern
      */
     @SuppressWarnings("unchecked") // Because of (Stream<Quad>) -- but why is that needed?
-    protected Stream<Quad> _find(final Node first, final Node second, final Node third, final Node fourth) {
+    private QuadOperator<Node, Stream<Quad>> find = (first, second, third, fourth) -> {
         debug("Querying on four-tuple pattern: {} {} {} {} .", first, second, third, fourth);
         final FourTupleMap fourTuples = local().get();
         if (isConcrete(first)) {
@@ -95,55 +117,60 @@ public abstract class PMapQuadTable extends PMapTupleTable<FourTupleMap, Quad>im
                             return twoTuples.get(third).map(oneTuples -> {
                                 if (isConcrete(fourth)) {
                                     debug("Using a specific fourth slot value.");
-                                    return oneTuples
-                                        .contains(fourth) ? of(quad(first, second, third, fourth)) : empty();
+                                    return oneTuples.contains(fourth) ? of(unmap(first, second, third, fourth))
+                                            : empty();
                                 }
                                 debug("Using a wildcard fourth slot value.");
-                                return oneTuples.stream().map(slot4 -> quad(first, second, third, slot4));
+                                return oneTuples.stream().map(slot4 -> unmap(first, second, third, slot4));
                             }).orElse(empty());
 
                         }
                         debug("Using wildcard third and fourth slot values.");
                         return twoTuples.flatten((slot3, oneTuples) -> oneTuples.stream()
-                                                 .map(slot4 -> quad(first, second, slot3, slot4)));
+                                .map(slot4 -> unmap(first, second, slot3, slot4)));
                     }).orElse(empty());
                 }
                 debug("Using wildcard second, third and fourth slot values.");
                 return threeTuples.flatten((slot2, twoTuples) -> twoTuples.flatten(
-                                                                                   (slot3, oneTuples) -> oneTuples.stream().map(slot4 -> quad(first, slot2, slot3, slot4))));
+                        (slot3, oneTuples) -> oneTuples.stream().map(slot4 -> unmap(first, slot2, slot3, slot4))));
             }).orElse(empty());
         }
         debug("Using a wildcard for all slot values.");
         return fourTuples.flatten((slot1, threeTuples) -> threeTuples.flatten((slot2, twoTuples) -> twoTuples
-                                                                              .flatten((slot3, oneTuples) -> oneTuples.stream().map(slot4 -> quad(slot1, slot2, slot3, slot4)))));
-    }
+                .flatten((slot3, oneTuples) -> oneTuples.stream().map(slot4 -> unmap(slot1, slot2, slot3, slot4)))));
+    };
+    
+    @Override
+    protected Consumer4<Node> add() {
+        return (first, second, third, fourth) -> {
+            debug("Adding four-tuple: {} {} {} {} .", first, second, third, fourth);
+            final FourTupleMap fourTuples = local().get();
+            ThreeTupleMap threeTuples = fourTuples.get(first).orElse(new ThreeTupleMap());
+            TwoTupleMap twoTuples = threeTuples.get(second).orElse(new TwoTupleMap());
+            PersistentSet<Node> oneTuples = twoTuples.get(third).orElse(PersistentSet.empty());
 
-    protected void _add(final Node first, final Node second, final Node third, final Node fourth) {
-        debug("Adding four-tuple: {} {} {} {} .", first, second, third, fourth);
-        final FourTupleMap fourTuples = local().get();
-        ThreeTupleMap threeTuples = fourTuples.get(first).orElse(new ThreeTupleMap());
-        TwoTupleMap twoTuples = threeTuples.get(second).orElse(new TwoTupleMap());
-        PersistentSet<Node> oneTuples = twoTuples.get(third).orElse(PersistentSet.empty());
-
-        if (!oneTuples.contains(fourth)) oneTuples = oneTuples.plus(fourth);
-        twoTuples = twoTuples.minus(third).plus(third, oneTuples);
-        threeTuples = threeTuples.minus(second).plus(second, twoTuples);
-        debug("Setting transactional index to new value.");
-        local().set(fourTuples.minus(first).plus(first, threeTuples));
+            if (!oneTuples.contains(fourth)) oneTuples = oneTuples.plus(fourth);
+            twoTuples = twoTuples.minus(third).plus(third, oneTuples);
+            threeTuples = threeTuples.minus(second).plus(second, twoTuples);
+            debug("Setting transactional index to new value.");
+            local().set(fourTuples.minus(first).plus(first, threeTuples));
+        };
     }
 
-    protected void _delete(final Node first, final Node second, final Node third, final Node fourth) {
-        debug("Removing four-tuple: {} {} {} {} .", first, second, third, fourth);
-        final FourTupleMap fourTuples = local().get();
-        fourTuples.get(first).ifPresent(threeTuples -> threeTuples.get(second)
-                                        .ifPresent(twoTuples -> twoTuples.get(third).ifPresent(oneTuples -> {
-                                            if (oneTuples.contains(fourth)) {
-                                                oneTuples = oneTuples.minus(fourth);
-                                                final TwoTupleMap newTwoTuples = twoTuples.minus(third).plus(third, oneTuples);
-                                                final ThreeTupleMap newThreeTuples = threeTuples.minus(second).plus(second, newTwoTuples);
-                                                debug("Setting transactional index to new value.");
-                                                local().set(fourTuples.minus(first).plus(first, newThreeTuples));
-                                            }
-                                        })));
+    protected Consumer4<Node> delete() {
+        return (first, second, third, fourth) -> {
+            debug("Removing four-tuple: {} {} {} {} .", first, second, third, fourth);
+            final FourTupleMap fourTuples = local().get();
+            fourTuples.get(first).ifPresent(threeTuples -> threeTuples.get(second)
+                    .ifPresent(twoTuples -> twoTuples.get(third).ifPresent(oneTuples -> {
+                if (oneTuples.contains(fourth)) {
+                    oneTuples = oneTuples.minus(fourth);
+                    final TwoTupleMap newTwoTuples = twoTuples.minus(third).plus(third, oneTuples);
+                    final ThreeTupleMap newThreeTuples = threeTuples.minus(second).plus(second, newTwoTuples);
+                    debug("Setting transactional index to new value.");
+                    local().set(fourTuples.minus(first).plus(first, newThreeTuples));
+                }
+            })));
+        };
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTripleTable.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTripleTable.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTripleTable.java
index dcb83ef..cbf9a31 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTripleTable.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTripleTable.java
@@ -25,6 +25,9 @@ import java.util.stream.Stream;
 
 import org.apache.jena.atlas.lib.persistent.PMap;
 import org.apache.jena.atlas.lib.persistent.PersistentSet;
+import org.apache.jena.atlas.lib.tuple.TriConsumer.Consumer3;
+import org.apache.jena.atlas.lib.tuple.TriFunction.TriOperator;
+import org.apache.jena.atlas.lib.tuple.TupleMap;
 import org.apache.jena.graph.Node;
 import org.apache.jena.graph.Triple;
 import org.apache.jena.sparql.core.mem.FourTupleMap.ThreeTupleMap;
@@ -35,8 +38,30 @@ import org.slf4j.Logger;
  * A {@link TripleTable} employing persistent maps to index triples in one particular slot order (e.g. SPO, OSP or POS).
  *
  */
-public abstract class PMapTripleTable extends PMapTupleTable<ThreeTupleMap, Triple>implements TripleTable {
+public class PMapTripleTable extends PMapTupleTable<ThreeTupleMap, Triple, Consumer3<Node>>implements TripleTable {
 
+    private static class TripleTupleMap extends TupleMap {
+
+        private TripleTupleMap(final String mappedOrder) {
+            super(mappedOrder, compileMapping("SPO", mappedOrder));
+        }
+    }
+    
+    /**
+     * @param order a name and order for this table
+     */
+    public PMapTripleTable(final String order) {
+        this(order, new TripleTupleMap(order));
+    }
+    
+    /**
+     * @param tableName a name for this table
+     * @param order the order of elements in this table
+     */
+    public PMapTripleTable(final String tableName, final TupleMap order) {
+        super(tableName, order);
+    }
+    
     private final static Logger log = getLogger(PMapTripleTable.class);
 
     @Override
@@ -48,14 +73,23 @@ public abstract class PMapTripleTable extends PMapTupleTable<ThreeTupleMap, Trip
     protected ThreeTupleMap initial() {
         return new ThreeTupleMap();
     }
-
-    /**
-     * @param tableName a name for this table
-     */
-    public PMapTripleTable(final String tableName) {
-        super(tableName);
+    
+    
+    @Override
+    public void add(final Triple t) {
+        map(add()).accept(t);
     }
 
+    @Override
+    public void delete(final Triple t) {
+        map(delete()).accept(t);
+    }
+    
+    @Override
+    public Stream<Triple> find(final Node s, final Node p, final Node o) {
+        return map(find).apply(s, p, o);
+    }
+    
     /**
      * We descend through the nested {@link PMap}s building up {@link Stream}s of partial tuples from which we develop a
      * {@link Stream} of full tuples which is our result. Use {@link Node#ANY} or <code>null</code> for a wildcard.
@@ -66,7 +100,7 @@ public abstract class PMapTripleTable extends PMapTupleTable<ThreeTupleMap, Trip
      * @return a <code>Stream</code> of tuples matching the pattern
      */
     @SuppressWarnings("unchecked") // Because of (Stream<Triple>) -- but why is that needed?
-    public Stream<Triple> _find(final Node first, final Node second, final Node third) {
+    private TriOperator<Node, Stream<Triple>> find = (first, second, third) -> {
         debug("Querying on three-tuple pattern: {} {} {} .", first, second, third);
         final ThreeTupleMap threeTuples = local().get();
         if (isConcrete(first)) {
@@ -77,53 +111,48 @@ public abstract class PMapTripleTable extends PMapTupleTable<ThreeTupleMap, Trip
                     return twoTuples.get(second).map(oneTuples -> {
                         if (isConcrete(third)) {
                             debug("Using a specific third slot value.");
-                            return oneTuples.contains(third) ? Stream.of(triple(first, second, third)) : empty();
+                            return oneTuples.contains(third) ? Stream.of(unmap(first, second, third)) : empty();
                         }
                         debug("Using a wildcard third slot value.");
-                        return oneTuples.stream().map(slot3 -> triple(first, second, slot3));
+                        return oneTuples.stream().map(slot3 -> unmap(first, second, slot3));
                     }).orElse(empty());
                 }
                 debug("Using wildcard second and third slot values.");
                 return twoTuples
-                    .flatten((slot2, oneTuples) -> oneTuples.stream().map(slot3 -> triple(first, slot2, slot3)));
+                    .flatten((slot2, oneTuples) -> oneTuples.stream().map(slot3 -> unmap(first, slot2, slot3)));
             }).orElse(empty());
         }
         debug("Using a wildcard for all slot values.");
         return threeTuples.flatten((slot1, twoTuples) -> twoTuples
-                                   .flatten((slot2, oneTuples) -> oneTuples.stream().map(slot3 -> triple(slot1, slot2, slot3))));
-    }
-
-    /**
-     * Constructs a {@link Triple} from the nodes given, using the appropriate order for this table. E.g. a POS table
-     * should return a {@code Triple} using ({@code second}, {@code third}, {@code first}).
-     *
-     * @param first
-     * @param second
-     * @param third
-     * @return a {@code Triple}
-     */
-    protected abstract Triple triple(final Node first, final Node second, final Node third);
-
-    protected void _add(final Node first, final Node second, final Node third) {
-        debug("Adding three-tuple {} {} {}", first, second, third);
-        final ThreeTupleMap threeTuples = local().get();
-        TwoTupleMap twoTuples = threeTuples.get(first).orElse(new TwoTupleMap());
-        PersistentSet<Node> oneTuples = twoTuples.get(second).orElse(PersistentSet.empty());
+                                   .flatten((slot2, oneTuples) -> oneTuples.stream().map(slot3 -> unmap(slot1, slot2, slot3))));
+    };
+    
+    @Override
+    protected Consumer3<Node> add() {
+        return (first, second, third) -> {
+            debug("Adding three-tuple {} {} {}", first, second, third);
+            final ThreeTupleMap threeTuples = local().get();
+            TwoTupleMap twoTuples = threeTuples.get(first).orElse(new TwoTupleMap());
+            PersistentSet<Node> oneTuples = twoTuples.get(second).orElse(PersistentSet.empty());
 
-        oneTuples = oneTuples.plus(third);
-        twoTuples = twoTuples.minus(second).plus(second, oneTuples);
-        local().set(threeTuples.minus(first).plus(first, twoTuples));
+            oneTuples = oneTuples.plus(third);
+            twoTuples = twoTuples.minus(second).plus(second, oneTuples);
+            local().set(threeTuples.minus(first).plus(first, twoTuples));
+        };
     }
-
-    protected void _delete(final Node first, final Node second, final Node third) {
-        debug("Deleting three-tuple {} {} {}", first, second, third);
-        final ThreeTupleMap threeTuples = local().get();
-        threeTuples.get(first).ifPresent(twoTuples -> twoTuples.get(second).ifPresent(oneTuples -> {
-            if (oneTuples.contains(third)) {
-                final TwoTupleMap newTwoTuples = twoTuples.minus(second).plus(second, oneTuples.minus(third));
-                debug("Setting transactional index to new value.");
-                local().set(threeTuples.minus(first).plus(first, newTwoTuples));
-            }
-        }));
+    
+    @Override
+    protected Consumer3<Node> delete() {
+        return (first, second, third) -> {
+            debug("Deleting three-tuple {} {} {}", first, second, third);
+            final ThreeTupleMap threeTuples = local().get();
+            threeTuples.get(first).ifPresent(twoTuples -> twoTuples.get(second).ifPresent(oneTuples -> {
+                if (oneTuples.contains(third)) {
+                    final TwoTupleMap newTwoTuples = twoTuples.minus(second).plus(second, oneTuples.minus(third));
+                    debug("Setting transactional index to new value.");
+                    local().set(threeTuples.minus(first).plus(first, newTwoTuples));
+                }
+            }));
+        };
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTupleTable.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTupleTable.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTupleTable.java
index 33d81c7..3231f2d 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTupleTable.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/PMapTupleTable.java
@@ -22,6 +22,7 @@ import static java.lang.ThreadLocal.withInitial;
 
 import java.util.concurrent.atomic.AtomicReference;
 
+import org.apache.jena.atlas.lib.tuple.TupleMap;
 import org.apache.jena.graph.Node;
 import org.apache.jena.query.ReadWrite;
 import org.slf4j.Logger;
@@ -31,8 +32,10 @@ import org.slf4j.Logger;
  *
  * @param <TupleMapType> the type of the internal structure holding table data
  * @param <TupleType> the type of tuple in which a subclass of this class transacts
+ * @param <ConsumerType> a type of consumer that can accept as many elements as exist in {@code TupleType}
  */
-public abstract class PMapTupleTable<TupleMapType, TupleType> implements TupleTable<TupleType> {
+public abstract class PMapTupleTable<TupleMapType, TupleType, ConsumerType>
+        extends OrderedTupleTable<TupleType, ConsumerType> implements TupleTable<TupleType> {
 
     /**
      * This method should always return the same value, but note that the same value may not necessarily be the same
@@ -41,7 +44,7 @@ public abstract class PMapTupleTable<TupleMapType, TupleType> implements TupleTa
      * @return a value to which to initialize the master table data.
      */
     protected abstract TupleMapType initial();
-
+    
     private final AtomicReference<TupleMapType> master = new AtomicReference<>(initial());
 
     /**
@@ -66,7 +69,8 @@ public abstract class PMapTupleTable<TupleMapType, TupleType> implements TupleTa
     /**
      * @param n a name for this table
      */
-    public PMapTupleTable(final String n) {
+    public PMapTupleTable(final String n, final TupleMap order) {
+        super(order);
         this.tableName = n;
     }
 
@@ -80,10 +84,11 @@ public abstract class PMapTupleTable<TupleMapType, TupleType> implements TupleTa
             log().debug(tableName + ": " + msg, values);
     }
 
+    /**
+     * {@link #local} is initialized via {@link #initial()}
+     */
     @Override
-    public void begin(final ReadWrite rw) {
-        // local is never used.
-    }
+    public void begin(final ReadWrite rw) {}
 
     @Override
     public void end() {

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTable.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTable.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTable.java
index c4fb521..9d9f8f0 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTable.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTable.java
@@ -1,19 +1,14 @@
 /*
- * 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
+ * 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
+ * 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.
+ * 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.core.mem;
@@ -23,7 +18,9 @@ import static org.apache.jena.graph.Node.ANY;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.stream.Stream;
-
+import org.apache.jena.atlas.lib.tuple.QuadConsumer.Consumer4;
+import org.apache.jena.atlas.lib.tuple.QuadFunction.QuadOperator;
+import org.apache.jena.atlas.lib.tuple.TupleMap;
 import org.apache.jena.graph.Node;
 import org.apache.jena.graph.Triple;
 import org.apache.jena.sparql.core.Quad;
@@ -31,7 +28,6 @@ import org.apache.jena.sparql.core.Quad;
 /**
  * A simplex or multiplex table of {@link Quad}s. Implementations may wish to override {@link #listGraphNodes()} with a
  * more efficient implementation.
- *
  */
 public interface QuadTable extends TupleTable<Quad> {
 
@@ -44,7 +40,7 @@ public interface QuadTable extends TupleTable<Quad> {
      * @param o the object node of the pattern
      * @return an {@link Stream} of matched quads
      */
-    Stream<Quad> find(Node g, Node s, Node p, Node o);
+    Stream<Quad> find(final Node g, final Node s, final Node p, final Node o);
 
     /**
      * Discover the graphs named in the table
@@ -63,9 +59,41 @@ public interface QuadTable extends TupleTable<Quad> {
     default Stream<Quad> findInUnionGraph(final Node s, final Node p, final Node o) {
         final Set<Triple> seen = new HashSet<>();
         return find(ANY, s, p, o).sequential()
-            .filter(q -> !q.isDefaultGraph())
-            .map(Quad::asTriple)
-            .filter(seen::add)
-            .map(t -> Quad.create(Quad.unionGraph, t)) ;
+                .filter(q -> !q.isDefaultGraph())
+                .map(Quad::asTriple)
+                .filter(seen::add)
+                .map(t -> Quad.create(Quad.unionGraph, t));
+    }
+
+    default Quad unmap(final TupleMap order, final Node first, final Node second, final Node third, final Node fourth) {
+        return apply(order, first, second, third, fourth, Quad::new);
+    }
+
+    static <X, Z> Z apply(final TupleMap tupleMap, final X x1, final X x2, final X x3, final X x4,
+            final QuadOperator<X, Z> f) {
+        final X x1a = get(tupleMap.mapIdx(0), x1, x2, x3, x4);
+        final X x2a = get(tupleMap.mapIdx(1), x1, x2, x3, x4);
+        final X x3a = get(tupleMap.mapIdx(2), x1, x2, x3, x4);
+        final X x4a = get(tupleMap.mapIdx(3), x1, x2, x3, x4);
+        return f.apply(x1a, x2a, x3a, x4a);
+    }
+
+    static <X> void accept(final TupleMap tupleMap, final X x1, final X x2, final X x3, final X x4,
+            final Consumer4<X> f) {
+        final X x1a = get(tupleMap.mapIdx(0), x1, x2, x3, x4);
+        final X x2a = get(tupleMap.mapIdx(1), x1, x2, x3, x4);
+        final X x3a = get(tupleMap.mapIdx(2), x1, x2, x3, x4);
+        final X x4a = get(tupleMap.mapIdx(3), x1, x2, x3, x4);
+        f.accept(x1a, x2a, x3a, x4a);
+    }
+
+    static <X> X get(final int i, final X x1, final X x2, final X x3, final X x4) {
+        switch (i) {
+        case 0: return x1;
+        case 1: return x2;
+        case 2: return x3;
+        case 3: return x4;
+        default: throw new IndexOutOfBoundsException("Quads have components 0, 1, 2, 3 but index = " + i + "!");
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTableForm.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTableForm.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTableForm.java
index 5bc09e6..014ad33 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTableForm.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/QuadTableForm.java
@@ -56,26 +56,6 @@ public enum QuadTableForm implements Supplier<QuadTable>,Predicate<Set<TupleSlot
             return new PMapQuadTable(name()) {
 
                 @Override
-                protected Quad quad(final Node g, final Node s, final Node p, final Node o) {
-                    return Quad.create(g, s, p, o);
-                }
-
-                @Override
-                public Stream<Quad> find(final Node g, final Node s, final Node p, final Node o) {
-                    return _find(g, s, p, o);
-                }
-
-                @Override
-                public void add(final Quad q) {
-                    _add(q.getGraph(), q.getSubject(), q.getPredicate(), q.getObject());
-                }
-
-                @Override
-                public void delete(final Quad q) {
-                    _delete(q.getGraph(), q.getSubject(), q.getPredicate(), q.getObject());
-                }
-
-                @Override
                 public Stream<Node> listGraphNodes() {
                     return local().get().entryStream().map(Entry::getKey);
                 }
@@ -86,34 +66,7 @@ public enum QuadTableForm implements Supplier<QuadTable>,Predicate<Set<TupleSlot
     /**
      * Graph-object-predicate-subject.
      */
-    GOPS(asList(GRAPH, OBJECT, PREDICATE, SUBJECT)) {
-        @Override
-        public PMapQuadTable get() {
-            return new PMapQuadTable(name()) {
-
-                @Override
-                protected Quad quad(final Node g, final Node o, final Node p, final Node s) {
-                    return Quad.create(g, s, p, o);
-                }
-
-                @Override
-                public Stream<Quad> find(final Node g, final Node s, final Node p, final Node o) {
-                    return _find(g, o, p, s);
-                }
-
-                @Override
-                public void add(final Quad q) {
-                    _add(q.getGraph(), q.getObject(), q.getPredicate(), q.getSubject());
-                }
-
-                @Override
-                public void delete(final Quad q) {
-                    _delete(q.getGraph(), q.getObject(), q.getPredicate(), q.getSubject());
-                }
-            };
-
-        }
-    },
+    GOPS(asList(GRAPH, OBJECT, PREDICATE, SUBJECT)),
 
     /**
      * Subject-predicate-object-graph.
@@ -124,32 +77,12 @@ public enum QuadTableForm implements Supplier<QuadTable>,Predicate<Set<TupleSlot
             return new PMapQuadTable(name()) {
 
                 @Override
-                protected Quad quad(final Node s, final Node p, final Node o, final Node g) {
-                    return Quad.create(g, s, p, o);
-                }
-
-                @Override
-                public Stream<Quad> find(final Node g, final Node s, final Node p, final Node o) {
-                    return _find(s, p, o, g);
-                }
-
-                @Override
                 public Stream<Quad> findInUnionGraph(final Node s, final Node p, final Node o) {
                     final AtomicReference<Triple> mostRecentlySeen = new AtomicReference<>();
                     return find(ANY, s, p, o).map(Quad::asTriple).filter(t->{
                         return !mostRecentlySeen.getAndSet(t).equals(t);
                     }).map(t->Quad.create(Quad.unionGraph, t)) ;
                 }
-
-                @Override
-                public void add(final Quad q) {
-                    _add(q.getSubject(), q.getPredicate(), q.getObject(), q.getGraph());
-                }
-
-                @Override
-                public void delete(final Quad q) {
-                    _delete(q.getSubject(), q.getPredicate(), q.getObject(), q.getGraph());
-                }
             };
         }
     },
@@ -157,64 +90,12 @@ public enum QuadTableForm implements Supplier<QuadTable>,Predicate<Set<TupleSlot
     /**
      * Object-subject-graph-predicate.
      */
-    OSGP(asList(OBJECT, SUBJECT, GRAPH, PREDICATE)) {
-        @Override
-        public PMapQuadTable get() {
-            return new PMapQuadTable(name()) {
-
-                @Override
-                protected Quad quad(final Node o, final Node s, final Node g, final Node p) {
-                    return Quad.create(g, s, p, o);
-                }
-
-                @Override
-                public Stream<Quad> find(final Node g, final Node s, final Node p, final Node o) {
-                    return _find(o, s, g, p);
-                }
-
-                @Override
-                public void add(final Quad q) {
-                    _add(q.getObject(), q.getSubject(), q.getGraph(), q.getPredicate());
-                }
-
-                @Override
-                public void delete(final Quad q) {
-                    _delete(q.getObject(), q.getSubject(), q.getGraph(), q.getPredicate());
-                }
-            };
-        }
-    },
+    OSGP(asList(OBJECT, SUBJECT, GRAPH, PREDICATE)),
 
     /**
      * Predicate-graph-subject-object.
      */
-    PGSO(asList(PREDICATE, GRAPH, SUBJECT, OBJECT)) {
-        @Override
-        public PMapQuadTable get() {
-            return new PMapQuadTable(name()) {
-
-                @Override
-                protected Quad quad(final Node p, final Node g, final Node s, final Node o) {
-                    return Quad.create(g, s, p, o);
-                }
-
-                @Override
-                public Stream<Quad> find(final Node g, final Node s, final Node p, final Node o) {
-                    return _find(p, g, s, o);
-                }
-
-                @Override
-                public void add(final Quad q) {
-                    _add(q.getPredicate(), q.getGraph(), q.getSubject(), q.getObject());
-                }
-
-                @Override
-                public void delete(final Quad q) {
-                    _delete(q.getPredicate(), q.getGraph(), q.getSubject(), q.getObject());
-                }
-            };
-        }
-    },
+    PGSO(asList(PREDICATE, GRAPH, SUBJECT, OBJECT)),
 
     /**
      * Object-predicate-subject-graph.
@@ -223,17 +104,7 @@ public enum QuadTableForm implements Supplier<QuadTable>,Predicate<Set<TupleSlot
         @Override
         public PMapQuadTable get() {
             return new PMapQuadTable(name()) {
-
-                @Override
-                protected Quad quad(final Node o, final Node p, final Node s, final Node g) {
-                    return Quad.create(g, s, p, o);
-                }
-
-                @Override
-                public Stream<Quad> find(final Node g, final Node s, final Node p, final Node o) {
-                    return _find(o, p, s, g);
-                }
-
+                
                 @Override
                 public Stream<Quad> findInUnionGraph(final Node s, final Node p, final Node o) {
                     final AtomicReference<Triple> mostRecentlySeen = new AtomicReference<>();
@@ -241,19 +112,14 @@ public enum QuadTableForm implements Supplier<QuadTable>,Predicate<Set<TupleSlot
                         return !mostRecentlySeen.getAndSet(t).equals(t);
                     }).map(t->Quad.create(Quad.unionGraph, t)) ;
                 }
-
-                @Override
-                public void add(final Quad q) {
-                    _add(q.getObject(), q.getPredicate(), q.getSubject(), q.getGraph());
-                }
-
-                @Override
-                public void delete(final Quad q) {
-                    _delete(q.getObject(), q.getPredicate(), q.getSubject(), q.getGraph());
-                }
             };
         }
     };
+    
+    @Override
+    public PMapQuadTable get() {
+        return new PMapQuadTable(name());
+    }
 
     private QuadTableForm(final List<TupleSlot> fp) {
         this.fullpattern = fp;

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTable.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTable.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTable.java
index 0a3c354..069d73e 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTable.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTable.java
@@ -1,31 +1,27 @@
 /*
- * 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
+ * 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
+ * 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.
+ * 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.core.mem;
 
 import java.util.stream.Stream;
-
+import org.apache.jena.atlas.lib.tuple.TriFunction.TriOperator;
+import org.apache.jena.atlas.lib.tuple.TriConsumer.Consumer3;
+import org.apache.jena.atlas.lib.tuple.TupleMap;
 import org.apache.jena.graph.Node;
 import org.apache.jena.graph.Triple;
 
 /**
  * A simplex or multiplex table of {@link Triple}s.
- *
  */
 public interface TripleTable extends TupleTable<Triple> {
 
@@ -37,10 +33,37 @@ public interface TripleTable extends TupleTable<Triple> {
      * @param o the object node of the pattern
      * @return an {@link Stream} of matched triples
      */
-    Stream<Triple> find(Node s, Node p, Node o);
+    Stream<Triple> find(final Node s, final Node p, final Node o);
 
     @Override
     default void clear() {
         find(null, null, null).forEach(this::delete);
     }
+
+    static <X> void accept(final TupleMap ordering, final X x1, final X x2, final X x3, final Consumer3<X> c) {
+        final X x1a = get(ordering.mapIdx(0), x1, x2, x3);
+        final X x2a = get(ordering.mapIdx(1), x1, x2, x3);
+        final X x3a = get(ordering.mapIdx(2), x1, x2, x3);
+        c.accept(x1a, x2a, x3a);
+    }
+
+    static <X, Z> Z apply(final TupleMap ordering, final X x1, final X x2, final X x3, final TriOperator<X, Z> f) {
+        final X x1a = get(ordering.mapIdx(0), x1, x2, x3);
+        final X x2a = get(ordering.mapIdx(1), x1, x2, x3);
+        final X x3a = get(ordering.mapIdx(2), x1, x2, x3);
+        return f.apply(x1a, x2a, x3a);
+    }
+
+    default Triple unmap(final TupleMap order, final Node first, final Node second, final Node third) {
+        return apply(order, first, second, third, Triple::new);
+    }
+
+    static <X> X get(final int i, final X x1, final X x2, final X x3) {
+        switch (i) {
+        case 0: return x1;
+        case 1: return x2;
+        case 2: return x3;
+        default: throw new IndexOutOfBoundsException("Triples have components 0, 1, 2 but index = " + i + "!");
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTableForm.java
----------------------------------------------------------------------
diff --git a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTableForm.java b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTableForm.java
index 822ff8c..031a75f 100644
--- a/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTableForm.java
+++ b/jena-arq/src/main/java/org/apache/jena/sparql/core/mem/TripleTableForm.java
@@ -27,115 +27,34 @@ import java.util.function.Predicate;
 import java.util.function.Supplier;
 import java.util.stream.Stream;
 
-import org.apache.jena.graph.Node;
-import org.apache.jena.graph.Triple;
-
 /**
  * Forms for triple indexes.
  *
  */
-public enum TripleTableForm implements Supplier<TripleTable>,Predicate<Set<TupleSlot>> {
+public enum TripleTableForm implements Supplier<TripleTable>, Predicate<Set<TupleSlot>> {
 
     /**
      * Subject-predicate-object.
      */
-    SPO(of(SUBJECT, PREDICATE), SUBJECT) {
-        @Override
-        public TripleTable get() {
-            return new PMapTripleTable(name()) {
-
-                @Override
-                protected Triple triple(final Node s, final Node p, final Node o) {
-                    return Triple.create(s, p, o);
-                }
-
-                @Override
-                public Stream<Triple> find(final Node s, final Node p, final Node o) {
-                    return _find(s, p, o);
-                }
-
-                @Override
-                public void add(final Triple t) {
-                    _add(t.getSubject(), t.getPredicate(), t.getObject());
-                }
-
-                @Override
-                public void delete(final Triple t) {
-                    _delete(t.getSubject(), t.getPredicate(), t.getObject());
-                }
-
-            };
-        }
-
-    },
+    SPO(of(SUBJECT, PREDICATE), SUBJECT),
     /**
      * Predicate-object-subject.
      */
-    POS(of(PREDICATE, OBJECT), PREDICATE) {
-
-        @Override
-        public TripleTable get() {
-            return new PMapTripleTable(name()) {
-
-                @Override
-                protected Triple triple(final Node p, final Node o, final Node s) {
-                    return Triple.create(s, p, o);
-                }
-
-                @Override
-                public Stream<Triple> find(final Node s, final Node p, final Node o) {
-                    return _find(p, o, s);
-                }
-
-                @Override
-                public void add(final Triple t) {
-                    _add(t.getPredicate(), t.getObject(), t.getSubject());
-                }
-
-                @Override
-                public void delete(final Triple t) {
-                    _delete(t.getPredicate(), t.getObject(), t.getSubject());
-                }
-
-            };
-        }
-    },
+    POS(of(PREDICATE, OBJECT), PREDICATE),
     /**
      * Object-subject-predicate.
      */
-    OSP(of(OBJECT, SUBJECT), OBJECT) {
-
-        @Override
-        public TripleTable get() {
-            return new PMapTripleTable(name()) {
-
-                @Override
-                protected Triple triple(final Node o, final Node s, final Node p) {
-                    return Triple.create(s, p, o);
-                }
-
-                @Override
-                public Stream<Triple> find(final Node s, final Node p, final Node o) {
-                    return _find(o, s, p);
-                }
-
-                @Override
-                public void add(final Triple t) {
-                    _add(t.getObject(), t.getSubject(), t.getPredicate());
-                }
-
-                @Override
-                public void delete(final Triple t) {
-                    _delete(t.getObject(), t.getSubject(), t.getPredicate());
-                }
-
-            };
-        }
-    };
+    OSP(of(OBJECT, SUBJECT), OBJECT);
+    
     private TripleTableForm(final Set<TupleSlot> tp, final TupleSlot op) {
         this.twoPrefix = tp;
         this.onePrefix = of(op);
     }
+    
+    @Override
+    public TripleTable get() {
+        return new PMapTripleTable(name());
+    }
 
     /**
      * Prefixes of the pattern for this table form.

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/QuadConsumer.java
----------------------------------------------------------------------
diff --git a/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/QuadConsumer.java b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/QuadConsumer.java
new file mode 100644
index 0000000..dfb8531
--- /dev/null
+++ b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/QuadConsumer.java
@@ -0,0 +1,42 @@
+/*
+ * 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.atlas.lib.tuple;
+
+import java.util.function.Consumer;
+
+/**
+ * Represents an operation that accepts four input arguments and returns no result. This is a four-arity specialization
+ * of {@link Consumer}. Unlike most other functional interfaces, {@code QuadConsumer} is expected to operate via
+ * side-effects.
+ * <p>
+ * This is a functional interface whose functional method is {@link #accept}.
+ *
+ * @param <W> the type of the first argument to the operation
+ * @param <X> the type of the second argument to the operation
+ * @param <Y> the type of the third argument to the operation
+ * @param <Z> the type of the fourth argument to the operation
+ * @see Consumer
+ */
+@FunctionalInterface
+public interface QuadConsumer<W, X, Y, Z> {
+
+    void accept(final W w, final X x, final Y y, final Z z);
+
+    /**
+     * A specialization of {@link QuadConsumer} in which all arguments are of the same type.
+     *
+     * @param <X> the type of all arguments
+     */
+    @FunctionalInterface
+    static interface Consumer4<X> extends QuadConsumer<X, X, X, X> {}
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/QuadFunction.java
----------------------------------------------------------------------
diff --git a/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/QuadFunction.java b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/QuadFunction.java
new file mode 100644
index 0000000..8bc8120
--- /dev/null
+++ b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/QuadFunction.java
@@ -0,0 +1,45 @@
+/*
+ * 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.atlas.lib.tuple;
+
+import java.util.function.Function;
+
+/**
+ * Represents a function that accepts four arguments and produces a result. This is a four-arity specialization of
+ * {@link Function}.
+ * <p>
+ * This is a functional interface whose functional method is {@link #apply}.
+ *
+ * @param <W> the type of the first argument to the function
+ * @param <X> the type of the second argument to the function
+ * @param <Y> the type of the third argument to the function
+ * @param <Z> the type of the fourth argument to the function
+ * @param <R> the type of the result of the function
+ * @see Function
+ */
+@FunctionalInterface
+public interface QuadFunction<W, X, Y, Z, R> {
+
+    R apply(final W w, final X x, final Y y, final Z z);
+
+    /**
+     * A specialization of {@link QuadFunction} in which all arguments are of the same type.
+     *
+     * @param <X> the type of all arguments
+     * @param <Z> the type of the result of the operation
+     */
+    @FunctionalInterface
+    static interface QuadOperator<X, Z> extends QuadFunction<X, X, X, X, Z> {}
+
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TriConsumer.java
----------------------------------------------------------------------
diff --git a/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TriConsumer.java b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TriConsumer.java
new file mode 100644
index 0000000..4c5bca1
--- /dev/null
+++ b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TriConsumer.java
@@ -0,0 +1,42 @@
+/*
+ * 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.atlas.lib.tuple;
+
+import java.util.function.Consumer;
+
+/**
+ * Represents an operation that accepts three input arguments and returns no result. This is a three-arity
+ * specialization of {@link Consumer}. Unlike most other functional interfaces, {@code TriConsumer} is expected to
+ * operate via side-effects.
+ * <p>
+ * This is a functional interface whose functional method is {@link #accept}.
+ *
+ * @param <X> the type of the first argument to the operation
+ * @param <Y> the type of the second argument to the operation
+ * @param <Z> the type of the third argument to the operation
+ * @see Consumer
+ */
+@FunctionalInterface
+public interface TriConsumer<X, Y, Z> {
+
+    void accept(final X x, final Y y, final Z z);
+
+    /**
+     * A specialization of {@link TriConsumer} in which all arguments are of the same type.
+     *
+     * @param <X> the type of all arguments
+     */
+    @FunctionalInterface
+    static interface Consumer3<X> extends TriConsumer<X, X, X> {}
+}

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TriFunction.java
----------------------------------------------------------------------
diff --git a/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TriFunction.java b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TriFunction.java
new file mode 100644
index 0000000..5329a29
--- /dev/null
+++ b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TriFunction.java
@@ -0,0 +1,42 @@
+/*
+ * 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.atlas.lib.tuple;
+
+import java.util.function.Function;
+
+/**
+ * Represents a function that accepts three arguments and produces a result. This is a three-arity specialization of
+ * {@link Function}.
+ * <p>
+ * This is a functional interface whose functional method is {@link #apply}.
+ *
+ * @param <X> the type of the first argument to the function
+ * @param <Y> the type of the second argument to the function
+ * @param <Z> the type of the second argument to the function
+ * @param <W> the type of the result of the function
+ * @see Function
+ */
+@FunctionalInterface
+public interface TriFunction<X, Y, Z, W> {
+    W apply(final X x, final Y y, final Z z);
+
+    /**
+     * A specialization of {@link TriFunction} in which all arguments are of the same type.
+     *
+     * @param <X> the type of all arguments
+     * @param <Z> the type of the result of the operation
+     */
+    @FunctionalInterface
+    static interface TriOperator<X, Z> extends TriFunction<X, X, X, Z> {}
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/jena/blob/f0666056/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TupleMap.java
----------------------------------------------------------------------
diff --git a/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TupleMap.java b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TupleMap.java
index bc4f69e..94ba8d2 100644
--- a/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TupleMap.java
+++ b/jena-base/src/main/java/org/apache/jena/atlas/lib/tuple/TupleMap.java
@@ -46,7 +46,6 @@ import org.apache.jena.atlas.lib.StrUtils ;
  * {@code 0<-1, 1<-2, 2<-0} 
  * and the "putTransform" is where to place the items: {@code 0->2, 1->0, 2->1}.
  */
-final
 public class TupleMap {
     /*
      * Naming.  getTransform (from src), putTransform(into dst)
@@ -117,7 +116,7 @@ public class TupleMap {
      * P->0, O->1) and not 1,2,0 (which is the extraction mapping). The label is
      * just a label and is not interpretted here.
      */
-    private TupleMap(String label, int... elements) {
+    protected TupleMap(String label, int... elements) {
         this.len = elements.length ; 
         this.label = label;
 
@@ -316,7 +315,7 @@ public class TupleMap {
     }
 
     /** Compile a mapping encoded as single charcaters e.g. "SPO", "POS" */
-    private static int[] compileMapping(String domain, String range) {
+    protected static int[] compileMapping(String domain, String range) {
         List<Character> input = StrUtils.toCharList(domain);
         List<Character> output = StrUtils.toCharList(range);
         return compileMapping(input, output);