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 2023/06/26 19:30:37 UTC

[jena] branch main updated: GH-1924: Add a "long" Turtle/TriG format variant

This is an automated email from the ASF dual-hosted git repository.

andy pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/jena.git


The following commit(s) were added to refs/heads/main by this push:
     new a8e80d7273 GH-1924: Add a "long" Turtle/TriG format variant
     new 3a41d91bc2 Merge pull request #1925 from rybesh/main
a8e80d7273 is described below

commit a8e80d7273da62d44381430f11850ff260172515
Author: Ryan Shaw <ry...@unc.edu>
AuthorDate: Mon Jun 26 12:17:13 2023 -0400

    GH-1924: Add a "long" Turtle/TriG format variant
    
    The "long" Turtle format variant is based on rdflib's "longturtle"
    format. It uses a fixed indentation width (2) and linebreaks after
    each sequence element.
---
 .../main/java/org/apache/jena/riot/RDFFormat.java  |   6 +
 .../org/apache/jena/riot/RDFWriterRegistry.java    |   6 +
 .../src/main/java/org/apache/jena/riot/RIOT.java   |   6 +
 .../writer/{WriterLib.java => IndentStyle.java}    |  33 +-
 .../apache/jena/riot/writer/TriGWriterLong.java    |  34 +
 .../org/apache/jena/riot/writer/TurtleShell.java   | 813 +++++++++++++--------
 .../apache/jena/riot/writer/TurtleWriterLong.java  |  34 +
 .../org/apache/jena/riot/writer/WriterLib.java     |  17 +
 .../org/apache/jena/riot/TestTurtleWriter.java     |   2 +
 .../apache/jena/riot/TestTurtleWriterPretty.java   |   2 +
 .../jena/riot/writer/TestRiotWriterGraph.java      |   2 +
 .../apache/jena/riot/writer/TestTurtleWriter.java  |   6 +-
 12 files changed, 643 insertions(+), 318 deletions(-)

diff --git a/jena-arq/src/main/java/org/apache/jena/riot/RDFFormat.java b/jena-arq/src/main/java/org/apache/jena/riot/RDFFormat.java
index 615a670b75..34c52c2097 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/RDFFormat.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/RDFFormat.java
@@ -33,6 +33,8 @@ public class RDFFormat {
     public static final RDFFormatVariant BLOCKS         = new RDFFormatVariant("blocks") ;
     /** Print out one per line */
     public static final RDFFormatVariant FLAT           = new RDFFormatVariant("flat") ;
+    /** Print with fixed indentation width and linebreaks after each sequence element */
+    public static final RDFFormatVariant LONG           = new RDFFormatVariant("long") ;
 
     /** Use ASCII output (N-triples, N-Quads) */
     public static final RDFFormatVariant ASCII          = new RDFFormatVariant("ascii") ;
@@ -51,6 +53,8 @@ public class RDFFormat {
     public static final RDFFormat        TURTLE_BLOCKS  = new RDFFormat(Lang.TURTLE, BLOCKS) ;
     /** Turtle - one line per triple  */
     public static final RDFFormat        TURTLE_FLAT    = new RDFFormat(Lang.TURTLE, FLAT) ;
+    /** Turtle - with fixed indentation width and linebreaks after each sequence element */
+    public static final RDFFormat        TURTLE_LONG    = new RDFFormat(Lang.TURTLE, LONG) ;
 
     /** N-Triples in UTF-8 */
     public static final RDFFormat        NTRIPLES_UTF8  = new RDFFormat(Lang.NTRIPLES, UTF8) ;
@@ -78,6 +82,8 @@ public class RDFFormat {
     public static final RDFFormat        TRIG_BLOCKS    = new RDFFormat(Lang.TRIG, BLOCKS) ;
     /** TriG - one line per triple  */
     public static final RDFFormat        TRIG_FLAT      = new RDFFormat(Lang.TRIG, FLAT) ;
+    /** TriG - with fixed indentation width and linebreaks after each sequence element */
+    public static final RDFFormat        TRIG_LONG      = new RDFFormat(Lang.TRIG, LONG) ;
 
     /** SHACL Compact Syntax */
     public static final RDFFormat        SHACLC         = new RDFFormat(Lang.SHACLC);
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterRegistry.java b/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterRegistry.java
index 5f9253e7ac..f23b04176c 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterRegistry.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/RDFWriterRegistry.java
@@ -68,6 +68,8 @@ public class RDFWriterRegistry
                 return new TurtleWriterBlocks() ;
             if ( Objects.equals(RDFFormat.TURTLE_FLAT, serialization) )
                 return new TurtleWriterFlat() ;
+            if ( Objects.equals(RDFFormat.TURTLE_LONG, serialization) )
+                return new TurtleWriterLong() ;
 
             if ( Objects.equals(RDFFormat.NTRIPLES_UTF8, serialization) )
                 return new NTriplesWriter() ;
@@ -97,6 +99,8 @@ public class RDFWriterRegistry
                 return new TriGWriterBlocks() ;
             if ( Objects.equals(RDFFormat.TRIG_FLAT, serialization) )
                 return new TriGWriterFlat() ;
+            if ( Objects.equals(RDFFormat.TRIG_LONG, serialization) )
+                return new TriGWriterLong() ;
             if ( Objects.equals(RDFFormat.NQUADS_UTF8, serialization) )
                 return new NQuadsWriter() ;
             if ( Objects.equals(RDFFormat.NQUADS_ASCII, serialization) )
@@ -159,6 +163,7 @@ public class RDFWriterRegistry
         register(RDFFormat.TURTLE_PRETTY,  wgfactory) ;
         register(RDFFormat.TURTLE_BLOCKS,  wgfactory) ;
         register(RDFFormat.TURTLE_FLAT,    wgfactory) ;
+        register(RDFFormat.TURTLE_LONG,    wgfactory) ;
 
         register(RDFFormat.NTRIPLES,       wgfactory) ;
         register(RDFFormat.NTRIPLES_ASCII, wgfactory) ;
@@ -213,6 +218,7 @@ public class RDFWriterRegistry
         register(RDFFormat.TRIG_PRETTY,    wgfactory) ;
         register(RDFFormat.TRIG_BLOCKS,    wgfactory) ;
         register(RDFFormat.TRIG_FLAT,      wgfactory) ;
+        register(RDFFormat.TRIG_LONG,      wgfactory) ;
 
         register(RDFFormat.NQUADS,         wgfactory) ;
         register(RDFFormat.NQUADS_ASCII,   wgfactory) ;
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/RIOT.java b/jena-arq/src/main/java/org/apache/jena/riot/RIOT.java
index 88c457e275..ce97434324 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/RIOT.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/RIOT.java
@@ -142,4 +142,10 @@ public class RIOT {
      * not output BASE even when given.
      */
     public static final Symbol symTurtleOmitBase = SystemARQ.allocSymbol(TURTLE_SYMBOL_BASE, "omitBase");
+
+
+    /**
+     * Printing style. Whether to use a "wide" or "long" indentation style.
+     */
+    public static final Symbol symTurtleIndentStyle = SystemARQ.allocSymbol(TURTLE_SYMBOL_BASE, "indentStyle");
 }
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/writer/WriterLib.java b/jena-arq/src/main/java/org/apache/jena/riot/writer/IndentStyle.java
similarity index 50%
copy from jena-arq/src/main/java/org/apache/jena/riot/writer/WriterLib.java
copy to jena-arq/src/main/java/org/apache/jena/riot/writer/IndentStyle.java
index 3adc22be85..b2d32e30b7 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/writer/WriterLib.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/writer/IndentStyle.java
@@ -18,29 +18,16 @@
 
 package org.apache.jena.riot.writer;
 
-import org.apache.jena.riot.RIOT;
-import org.apache.jena.sparql.util.Context;
-
-/** Package-scoped utilities */
-/*package*/ class WriterLib {
-
-    private static final DirectiveStyle dftDirectiveStyle = DirectiveStyle.AT;
-
-    // Determine the directive style (applies to PREFIX and BASE).
-    /*package*/ static DirectiveStyle directiveStyle(Context context) {
-        if ( context == null )
-            return dftDirectiveStyle;
-        Object x = context.get(RIOT.symTurtleDirectiveStyle) ;
-
-        if ( x instanceof String ) {
-            String s = (String)x ;
-            DirectiveStyle style = DirectiveStyle.create(s);
-            return style == null ? dftDirectiveStyle : style;
+enum IndentStyle {
+    WIDE, LONG ;
+    public static IndentStyle create(String label) {
+        String s = label.toLowerCase() ;
+        switch(s) {
+            case "wide":
+                return IndentStyle.WIDE ;
+            case "long":
+                return IndentStyle.LONG ;
         }
-        if ( x instanceof DirectiveStyle )
-            return (DirectiveStyle)x ;
-
-        // Default choice; includes null in context.
-        return dftDirectiveStyle;
+        return null;
     }
 }
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/writer/TriGWriterLong.java b/jena-arq/src/main/java/org/apache/jena/riot/writer/TriGWriterLong.java
new file mode 100644
index 0000000000..4f47fde35e
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/riot/writer/TriGWriterLong.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.riot.writer ;
+
+import org.apache.jena.atlas.io.IndentedWriter ;
+import org.apache.jena.riot.RIOT;
+import org.apache.jena.riot.system.PrefixMap ;
+import org.apache.jena.sparql.core.DatasetGraph ;
+import org.apache.jena.sparql.util.Context ;
+
+public class TriGWriterLong extends TriGWriter {
+
+    @Override
+    protected void output(IndentedWriter iOut, DatasetGraph dsg, PrefixMap prefixMap, String baseURI, Context context) {
+        context.set(RIOT.symTurtleIndentStyle, "long");
+        super.output(iOut, dsg, prefixMap, baseURI, context);
+    }
+}
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/writer/TurtleShell.java b/jena-arq/src/main/java/org/apache/jena/riot/writer/TurtleShell.java
index f7c23a911e..3e152065c0 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/writer/TurtleShell.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/writer/TurtleShell.java
@@ -58,6 +58,7 @@ public abstract class TurtleShell {
     protected final String         baseURI ;
     protected final Context        context ;
     protected final DirectiveStyle    prefixStyle;
+    protected final IndentStyle    indentStyle;
 
     protected TurtleShell(IndentedWriter out, PrefixMap pmap, String baseURI, NodeFormatter nodeFmt, Context context) {
         this.out = out ;
@@ -68,6 +69,7 @@ public abstract class TurtleShell {
         this.nodeFmt = nodeFmt ;
         this.context = context;
         this.prefixStyle = WriterLib.directiveStyle(context) ;
+        this.indentStyle = WriterLib.indentStyle(context) ;
     }
 
     protected TurtleShell(IndentedWriter out, PrefixMap pmap, String baseURI, Context context) {
@@ -92,7 +94,7 @@ public abstract class TurtleShell {
 
     /** Write graph in Turtle syntax (or part of TriG) */
     protected void writeGraphTTL(Graph graph) {
-        ShellGraph x = new ShellGraph(graph, null, null, null) ;
+        ShellGraph x = createShellGraph(graph, null, null, null) ;
         x.writeGraph() ;
     }
 
@@ -101,13 +103,504 @@ public abstract class TurtleShell {
         Graph g = (graphName == null || Quad.isDefaultGraph(graphName))
             ? dsg.getDefaultGraph()
             : dsg.getGraph(graphName) ;
-        ShellGraph x = new ShellGraph(g, graphName, dsg, graphNames) ;
+        ShellGraph x = createShellGraph(g, graphName, dsg, graphNames) ;
         x.writeGraph() ;
     }
 
+    private ShellGraph createShellGraph(Graph graph, Node graphName, DatasetGraph dsg, Set<Node> graphNames) {
+        if (this.indentStyle == IndentStyle.LONG) {
+            return new ShellGraphLong(graph, graphName, dsg, graphNames) ;
+        } else {
+            return new ShellGraphWide(graph, graphName, dsg, graphNames) ;
+        }
+    }
+
+    private final class ShellGraphWide extends ShellGraph {
+
+         private ShellGraphWide(Graph graph, Node graphName, DatasetGraph dsg, Set<Node> graphNames) {
+            super(graph, graphName, dsg, graphNames);
+        }
+
+         protected boolean writeRemainingNLinkedLists(boolean somethingWritten) {
+            // Print carefully - need a label for the first cell.
+            // So we write out the first element of the list in triples, then
+            // put the remainer as a pretty list
+            for ( Node n : nLinkedLists.keySet() ) {
+                if ( somethingWritten )
+                    out.println() ;
+                somethingWritten = true ;
+
+                List<Node> x = nLinkedLists.get(n) ;
+                writeNode(n) ;
+
+                write_S_P_Gap();
+                out.pad() ;
+
+                writeNode(RDF_First) ;
+                print(" ") ;
+                writeNode(x.get(0)) ;
+                print(" ;") ;
+                println() ;
+                writeNode(RDF_Rest) ;
+                print("  ") ;
+                x = x.subList(1, x.size()) ;
+                writeList(x) ;
+                print(" .") ;
+                out.decIndent(INDENT_PREDICATE) ;
+                println() ;
+            }
+            return somethingWritten ;
+        }
+
+        protected void writeCluster(Node subject, Collection<Triple> cluster) {
+            if ( cluster.isEmpty() )
+                return ;
+            writeNode(subject) ;
+            writeClusterPredicateObjectList(INDENT_PREDICATE, cluster) ;
+        }
+
+        protected void writePredicateObjectList(Collection<Triple> cluster) {
+            Map<Node, List<Node>> pGroups = groupByPredicates(cluster) ;
+            Collection<Node> predicates = pGroups.keySet() ;
+
+            // Find longest predicate URI
+            int predicateMaxWidth = Widths.calcWidth(prefixMap, baseURI, predicates, MIN_PREDICATE, LONG_PREDICATE, printTypeKeyword) ;
+
+            boolean first = true ;
+
+            if ( !OBJECT_LISTS ) {
+                for ( Node p : predicates ) {
+                    for ( Node o : pGroups.get(p) ) {
+                        writePredicateObject(p, o, predicateMaxWidth, first) ;
+                        first = false ;
+                    }
+                }
+                return ;
+            }
+
+            for ( Node p : predicates ) {
+                // Literals in the group
+                List<Node> rdfLiterals = new ArrayList<>() ;
+                // Non-literals, printed
+                List<Node> rdfSimpleNodes = new ArrayList<>() ;
+                // Non-literals, printed (), or []-embedded
+                List<Node> rdfComplexNodes = new ArrayList<>() ;
+
+                for ( Node o : pGroups.get(p) ) {
+                    if ( o.isLiteral() ) {
+                        rdfLiterals.add(o) ;
+                        continue ;
+                    }
+                    if ( isPrettyNode(o) ) {
+                        rdfComplexNodes.add(o) ;
+                        continue ;
+                    }
+                    rdfSimpleNodes.add(o) ;
+                }
+
+                if ( ! rdfLiterals.isEmpty() ) {
+                    writePredicateObjectList(p, rdfLiterals, predicateMaxWidth, first) ;
+                    first = false ;
+                }
+                if ( ! rdfSimpleNodes.isEmpty() ) {
+                    writePredicateObjectList(p, rdfSimpleNodes, predicateMaxWidth, first) ;
+                    first = false ;
+                }
+                for ( Node o : rdfComplexNodes ) {
+                    writePredicateObject(p, o, predicateMaxWidth, first) ;
+                    first = false ;
+                }
+            }
+        }
+
+        private void writePredicateObjectList(Node p, List<Node> objects, int predicateMaxWidth, boolean first) {
+           writePredicate(p, predicateMaxWidth, first) ;
+            out.incIndent(INDENT_OBJECT) ;
+
+            boolean lastObjectMultiLine = false ;
+            boolean firstObject = true ;
+            for ( Node o : objects ) {
+                if ( !firstObject ) {
+                    if ( out.getCurrentOffset() > 0 )
+                        out.print(" , ") ;
+                    else
+                        // Before the current indent, due to a multiline literal being written raw.
+                        // We will pad spaces to indent on output spaces.  Don't add a first " "
+                        out.print(", ") ;
+                }
+                else
+                    firstObject = false ;
+                int row1 = out.getRow() ;
+                writeNode(o) ;
+                int row2 = out.getRow();
+                lastObjectMultiLine = (row2 > row1) ;
+            }
+            out.decIndent(INDENT_OBJECT) ;
+        }
+
+        private void writePredicateObject(Node p, Node obj, int predicateMaxWidth, boolean first) {
+            writePredicate(p, predicateMaxWidth, first) ;
+            out.incIndent(INDENT_OBJECT) ;
+            writeNodePretty(obj) ;
+            out.decIndent(INDENT_OBJECT) ;
+        }
+
+        private void writePredicate(Node p, int predicateMaxWidth, boolean first) {
+            if ( ! first ) {
+                print(" ;") ;
+                println() ;
+            }
+            int colPredicateStart = out.getAbsoluteIndent() ;
+
+            if ( printTypeKeyword && RDF_type.equals(p) )
+                print("a") ;
+            else
+                writeNode(p) ;
+
+            int colPredicateFinish = out.getCol() ;
+            int wPredicate = (colPredicateFinish - colPredicateStart) ;
+
+            if ( wPredicate > LONG_PREDICATE )
+                println() ;
+            else {
+                out.pad(predicateMaxWidth) ;
+                gap(GAP_P_O) ;
+            }
+        }
+
+        protected void writeNestedObjectTopLevel(Node subject) {
+            if ( true ) {
+                writeNestedObject(subject) ;
+                out.println(" .") ;
+            } else {
+                // Alternative.
+                Collection<Triple> cluster = triplesOfSubject(subject) ;
+                print("[]") ;
+                writeClusterPredicateObjectList(0, cluster) ;
+            }
+        }
+
+        protected void writeNestedObject(Node node) {
+            Collection<Triple> x = triplesOfSubject(node) ;
+
+            if ( x.isEmpty() ) {
+                print("[] ") ;
+                return ;
+            }
+
+            if ( isCompact(x) ) {
+                print("[ ") ;
+                out.incIndent(2) ;
+                writePredicateObjectList(x) ;
+                out.decIndent(2) ;
+                print(" ]") ;
+                return ;
+            }
+
+            int indent0 = out.getAbsoluteIndent() ;
+            int here = out.getCol() ;
+            out.setAbsoluteIndent(here) ;
+            print("[ ") ;
+            out.incIndent(2) ;
+            writePredicateObjectList(x) ;
+            out.decIndent(2) ;
+            if ( true ) {
+                println() ; // Newline for "]"
+                print("]") ;
+            } else { // Compact
+                print(" ]") ;
+            }
+            out.setAbsoluteIndent(indent0) ;
+        }
+
+        protected void writeList(List<Node> elts) {
+            if ( elts.size() == 0 ) {
+                out.print("()") ;
+                return ;
+            }
+
+            if ( false ) {
+                out.print("(") ;
+                for ( Node n : elts ) {
+                    out.print(" ") ;
+                    writeNodePretty(n) ;
+                }
+                out.print(" )") ;
+            }
+
+            if ( true ) {
+                // "fresh line mode" means printed one on new line
+                // Multi line items are ones that can be multiple lines. Non-literals.
+                // Was the previous row a multiLine?
+                boolean lastItemFreshLine = false ;
+                // Have there been any items that causes "fresh line" mode?
+                boolean multiLineAny = false ;
+                boolean first = true ;
+
+                // Where we started.
+                int originalIndent = out.getAbsoluteIndent() ;
+                // Rebase indent here.
+                int x = out.getCol() ;
+                out.setAbsoluteIndent(x);
+
+                out.print("(") ;
+                out.incIndent(2);
+                for ( Node n : elts ) {
+
+                    // Print this item on a fresh line? (still to check: first line)
+                    boolean thisItemFreshLine = /* multiLineAny | */ n.isBlank() ;
+
+                    // Special case List in List.
+                    // Start on this line if last item was on this line.
+                    if ( lists.containsKey(n) )
+                        thisItemFreshLine = lastItemFreshLine ;
+
+                    // Starting point.
+                    if ( ! first ) {
+                        if ( lastItemFreshLine | thisItemFreshLine )
+                            out.println() ;
+                        else
+                            out.print(" ") ;
+                    }
+
+                    first = false ;
+                    //Literals with newlines: int x1 = out.getRow() ;
+                    // Adds INDENT_OBJECT even for a [ one triple ]
+                    // Special case [ one triple ]??
+                    writeNodePretty(n) ;
+                    //Literals with newlines:int x2 = out.getRow() ;
+                    //Literals with newlines: boolean multiLineAnyway = ( x1 != x2 ) ;
+                    lastItemFreshLine = thisItemFreshLine ;
+                    multiLineAny  = multiLineAny | thisItemFreshLine ;
+
+                }
+                if ( multiLineAny )
+                    out.println() ;
+                else
+                    out.print(" ") ;
+                out.decIndent(2);
+                out.setAbsoluteIndent(x);
+                out.print(")") ;
+                out.setAbsoluteIndent(originalIndent) ;
+            }
+        }
+
+        protected void write_S_P_Gap() {
+            if ( out.getCol() > LONG_SUBJECT )
+                out.println() ;
+            else
+                gap(GAP_S_P) ;
+        }
+    }
+
+    private final class ShellGraphLong extends ShellGraph {
+
+        private static final int INDENT = 2 ;
+
+        private ShellGraphLong(Graph graph, Node graphName, DatasetGraph dsg, Set<Node> graphNames) {
+            super(graph, graphName, dsg, graphNames);
+        }
+
+
+        protected boolean writeRemainingNLinkedLists(boolean somethingWritten) {
+            // Print carefully - need a label for the first cell.
+            // So we write out the first element of the list in triples, then
+            // put the remainer as a pretty list
+            for ( Node n : nLinkedLists.keySet() ) {
+                if ( somethingWritten )
+                    out.println() ;
+                somethingWritten = true ;
+
+                List<Node> x = nLinkedLists.get(n) ;
+                writeNode(n) ;
+
+                write_S_P_Gap();
+                out.pad() ;
+
+                writeNode(RDF_First) ;
+                print(" ") ;
+                writeNode(x.get(0)) ;
+                print(" ;") ;
+                println() ;
+                writeNode(RDF_Rest) ;
+                print("  ") ;
+                x = x.subList(1, x.size()) ;
+                writeList(x) ;
+                print(" .") ;
+                out.decIndent(INDENT) ;
+                println() ;
+            }
+            return somethingWritten ;
+        }
+
+        protected void writeCluster(Node subject, Collection<Triple> cluster) {
+            if ( cluster.isEmpty() )
+                return ;
+            writeNode(subject) ;
+            writeClusterPredicateObjectList(INDENT, cluster) ;
+        }
+        // Writing predicate-object lists.
+        // We group the cluster by predicate and within each group
+        // we print:
+        //    literals, then simple objects, then pretty objects
+
+        protected void writePredicateObjectList(Collection<Triple> cluster) {
+            Map<Node, List<Node>> pGroups = groupByPredicates(cluster) ;
+            Collection<Node> predicates = pGroups.keySet() ;
+
+            boolean first = true ;
+
+            if ( !OBJECT_LISTS ) {
+                for ( Node p : predicates ) {
+                    for ( Node o : pGroups.get(p) ) {
+                        writePredicateObject(p, o, first) ;
+                        first = false ;
+                    }
+                }
+                return ;
+            }
+
+            for ( Node p : predicates ) {
+                // Literals in the group
+                List<Node> rdfLiterals = new ArrayList<>() ;
+                // Non-literals, printed
+                List<Node> rdfSimpleNodes = new ArrayList<>() ;
+                // Non-literals, printed (), or []-embedded
+                List<Node> rdfComplexNodes = new ArrayList<>() ;
+
+                for ( Node o : pGroups.get(p) ) {
+                    if ( o.isLiteral() ) {
+                        rdfLiterals.add(o) ;
+                        continue ;
+                    }
+                    if ( isPrettyNode(o) ) {
+                        rdfComplexNodes.add(o) ;
+                        continue ;
+                    }
+                    rdfSimpleNodes.add(o) ;
+                }
+
+                if ( ! rdfLiterals.isEmpty() ) {
+                    writePredicateObjectList(p, rdfLiterals, first) ;
+                    first = false ;
+                }
+                if ( ! rdfSimpleNodes.isEmpty() ) {
+                    writePredicateObjectList(p, rdfSimpleNodes, first) ;
+                    first = false ;
+                }
+                for ( Node o : rdfComplexNodes ) {
+                    writePredicateObject(p, o, first) ;
+                    first = false ;
+                }
+            }
+        }
+
+        private void writePredicateObject(Node p, Node obj, boolean first) {
+            writePredicate(p, first) ;
+            writeNodePretty(obj) ;
+        }
+
+        private void writePredicateObjectList(Node p, List<Node> objects, boolean first) {
+            writePredicate(p, first) ;
+            out.incIndent(INDENT) ;
+
+            boolean firstObject = true ;
+            for ( Node o : objects ) {
+                if ( !firstObject ) {
+                    if ( out.getCurrentOffset() > 0 )
+                        out.print(" , ") ;
+                    else
+                        // Before the current indent, due to a multiline literal being written raw.
+                        // We will pad spaces to indent on output spaces.  Don't add a first " "
+                        out.print(", ") ;
+                }
+                else
+                    firstObject = false ;
+                writeNode(o) ;
+            }
+            out.decIndent(INDENT) ;
+        }
+
+        /** Write a predicate - jump to next line if deemed long */
+        private void writePredicate(Node p, boolean first) {
+            if ( ! first ) {
+                print(" ;") ;
+                println() ;
+            }
+            int colPredicateStart = out.getAbsoluteIndent() ;
+
+            if ( printTypeKeyword && RDF_type.equals(p) )
+                print("a") ;
+            else
+                writeNode(p) ;
+
+            int colPredicateFinish = out.getCol() ;
+            int wPredicate = (colPredicateFinish - colPredicateStart) ;
+
+            if ( wPredicate > LONG_PREDICATE )
+                println() ;
+            else {
+                gap(1) ;
+            }
+        }
+
+        protected void writeNestedObjectTopLevel(Node subject) {
+            writeNestedObject(subject) ;
+            out.println(" .") ;
+        }
+
+        protected void writeNestedObject(Node node) {
+            Collection<Triple> x = triplesOfSubject(node) ;
+
+            if ( x.isEmpty() ) {
+                print("[] ") ;
+                return ;
+            }
+
+            if ( isCompact(x) ) {
+                print("[ ") ;
+                out.incIndent(2) ;
+                writePredicateObjectList(x) ;
+                out.decIndent(2) ;
+                print(" ]") ;
+                return ;
+            }
+
+            print("[") ;
+            out.println();
+            out.incIndent(2) ;
+            writePredicateObjectList(x) ;
+            out.decIndent(2) ;
+            println() ;
+            print("]") ;
+        }
+
+        protected void writeList(List<Node> elts) {
+            if ( elts.size() == 0 ) {
+                out.print("()") ;
+                return ;
+            }
+
+            out.print("(") ;
+            out.incIndent(2);
+            for ( Node n : elts ) {
+                out.println() ;
+                writeNodePretty(n) ;
+            }
+            out.println() ;
+            out.decIndent(2);
+            out.print(")") ;
+        }
+
+        protected void write_S_P_Gap() {
+            out.println();
+        }
+    }
+
+
     // Write one graph - using an inner object class to isolate
     // the state variables for writing a single graph.
-    private final class ShellGraph {
+    private abstract class ShellGraph {
         // Dataset (for writing graphs in datasets) -- may be null
         private final DatasetGraph          dsg ;
         private final Collection<Node>      graphNames ;
@@ -123,13 +616,13 @@ public abstract class TurtleShell {
         private final Set<Node>             freeBnodes ;
 
         // The head node in each well-formed list -> list elements
-        private final Map<Node, List<Node>> lists ;
+        protected final Map<Node, List<Node>> lists ;
 
         // List that do not have any incoming triples
         private final Map<Node, List<Node>> freeLists ;
 
         // Lists that have more than one incoming triple
-        private final Map<Node, List<Node>> nLinkedLists ;
+        protected final Map<Node, List<Node>> nLinkedLists ;
 
         // All nodes that are part of list structures.
         private final Collection<Node>      listElts ;
@@ -138,7 +631,7 @@ public abstract class TurtleShell {
         // This is true for the main pretty printing then
         // false when we are clearing up unwritten triples.
         private boolean allowDeepPretty = true ;
-        private final boolean printTypeKeyword;
+        protected final boolean printTypeKeyword;
 
         private ShellGraph(Graph graph, Node graphName, DatasetGraph dsg, Set<Node> graphNames) {
             this.dsg = dsg ;
@@ -314,7 +807,7 @@ public abstract class TurtleShell {
         }
 
         /** Get triples with the same subject */
-        private Collection<Triple> triplesOfSubject(Node subj) {
+        protected Collection<Triple> triplesOfSubject(Node subj) {
             return RiotLib.triplesOfSubject(graph, subj) ;
         }
 
@@ -505,7 +998,7 @@ public abstract class TurtleShell {
 
         // ----
 
-        private void writeGraph() {
+        protected void writeGraph() {
             Iterator<Node> subjects = listSubjects() ;
             boolean somethingWritten = writeBySubject(subjects) ;
             // Write remainders
@@ -515,45 +1008,16 @@ public abstract class TurtleShell {
             // 2 - Free standing lists
             somethingWritten = writeRemainingFreeLists(somethingWritten) ;
 
-            // 3 - Blank nodes that are unwritten single objects.
-            //            System.err.println("## ## ##") ;
-            //            printDetails("nestedObjects", nestedObjects) ;
-            //            printDetails("nestedObjectsWritten", nestedObjectsWritten) ;
-            Set<Node> singleNodes = SetUtils.difference(nestedObjects, nestedObjectsWritten) ;
-            somethingWritten = writeRemainingNestedObjects(singleNodes, somethingWritten) ;
-        }
-
-        private boolean writeRemainingNLinkedLists(boolean somethingWritten) {
-            // Print carefully - need a label for the first cell.
-            // So we write out the first element of the list in triples, then
-            // put the remainer as a pretty list
-            for ( Node n : nLinkedLists.keySet() ) {
-                if ( somethingWritten )
-                    out.println() ;
-                somethingWritten = true ;
-
-                List<Node> x = nLinkedLists.get(n) ;
-                writeNode(n) ;
-
-                write_S_P_Gap();
-                out.pad() ;
-
-                writeNode(RDF_First) ;
-                print(" ") ;
-                writeNode(x.get(0)) ;
-                print(" ;") ;
-                println() ;
-                writeNode(RDF_Rest) ;
-                print("  ") ;
-                x = x.subList(1, x.size()) ;
-                writeList(x) ;
-                print(" .") ;
-                out.decIndent(INDENT_PREDICATE) ;
-                println() ;
-            }
-            return somethingWritten ;
+            // 3 - Blank nodes that are unwritten single objects.
+            //            System.err.println("## ## ##") ;
+            //            printDetails("nestedObjects", nestedObjects) ;
+            //            printDetails("nestedObjectsWritten", nestedObjectsWritten) ;
+            Set<Node> singleNodes = SetUtils.difference(nestedObjects, nestedObjectsWritten) ;
+            somethingWritten = writeRemainingNestedObjects(singleNodes, somethingWritten) ;
         }
 
+        protected abstract boolean writeRemainingNLinkedLists(boolean somethingWritten) ;
+
         // Write free standing lists - ones where the head is not an object of
         // some other triple. Turtle does not allow free standing (... ) .
         // so write as a predicateObjectList for one element.
@@ -660,16 +1124,11 @@ public abstract class TurtleShell {
         }
 
         // A Cluster is a collection of triples with the same subject.
-        private void writeCluster(Node subject, Collection<Triple> cluster) {
-            if ( cluster.isEmpty() )
-                return ;
-            writeNode(subject) ;
-            writeClusterPredicateObjectList(INDENT_PREDICATE, cluster) ;
-        }
+        protected abstract void writeCluster(Node subject, Collection<Triple> cluster) ;
 
         // Write the PredicateObjectList for a subject already output.
         // The subject may have been a "[]" or a URI - the indentation is passed in.
-        private void writeClusterPredicateObjectList(int indent, Collection<Triple> cluster) {
+        protected void writeClusterPredicateObjectList(int indent, Collection<Triple> cluster) {
             write_S_P_Gap() ;
             out.incIndent(indent) ;
             out.pad() ;
@@ -679,122 +1138,9 @@ public abstract class TurtleShell {
             println() ;
         }
 
-        // Writing predicate-object lists.
-        // We group the cluster by predicate and within each group
-        // we print:
-        //    literals, then simple objects, then pretty objects
-
-        private void writePredicateObjectList(Collection<Triple> cluster) {
-            Map<Node, List<Node>> pGroups = groupByPredicates(cluster) ;
-            Collection<Node> predicates = pGroups.keySet() ;
-
-            // Find longest predicate URI
-            int predicateMaxWidth = Widths.calcWidth(prefixMap, baseURI, predicates, MIN_PREDICATE, LONG_PREDICATE, printTypeKeyword) ;
-
-            boolean first = true ;
-
-            if ( !OBJECT_LISTS ) {
-                for ( Node p : predicates ) {
-                    for ( Node o : pGroups.get(p) ) {
-                        writePredicateObject(p, o, predicateMaxWidth, first) ;
-                        first = false ;
-                    }
-                }
-                return ;
-            }
-
-            for ( Node p : predicates ) {
-                // Literals in the group
-                List<Node> rdfLiterals = new ArrayList<>() ;
-                // Non-literals, printed
-                List<Node> rdfSimpleNodes = new ArrayList<>() ;
-                // Non-literals, printed (), or []-embedded
-                List<Node> rdfComplexNodes = new ArrayList<>() ;
-
-                for ( Node o : pGroups.get(p) ) {
-                    if ( o.isLiteral() ) {
-                        rdfLiterals.add(o) ;
-                        continue ;
-                    }
-                    if ( isPrettyNode(o) ) {
-                        rdfComplexNodes.add(o) ;
-                        continue ;
-                    }
-                    rdfSimpleNodes.add(o) ;
-                }
-
-                if ( ! rdfLiterals.isEmpty() ) {
-                    writePredicateObjectList(p, rdfLiterals, predicateMaxWidth, first) ;
-                    first = false ;
-                }
-                if ( ! rdfSimpleNodes.isEmpty() ) {
-                    writePredicateObjectList(p, rdfSimpleNodes, predicateMaxWidth, first) ;
-                    first = false ;
-                }
-                for ( Node o : rdfComplexNodes ) {
-                    writePredicateObject(p, o, predicateMaxWidth, first) ;
-                    first = false ;
-                }
-            }
-        }
-
-        private void writePredicateObject(Node p, Node obj, int predicateMaxWidth, boolean first) {
-            writePredicate(p, predicateMaxWidth, first) ;
-            out.incIndent(INDENT_OBJECT) ;
-            writeNodePretty(obj) ;
-            out.decIndent(INDENT_OBJECT) ;
-        }
-
-        private void writePredicateObjectList(Node p, List<Node> objects, int predicateMaxWidth, boolean first) {
-            writePredicate(p, predicateMaxWidth, first) ;
-            out.incIndent(INDENT_OBJECT) ;
-
-            boolean lastObjectMultiLine = false ;
-            boolean firstObject = true ;
-            for ( Node o : objects ) {
-                if ( !firstObject ) {
-                    if ( out.getCurrentOffset() > 0 )
-                        out.print(" , ") ;
-                    else
-                        // Before the current indent, due to a multiline literal being written raw.
-                        // We will pad spaces to indent on output spaces.  Don't add a first " "
-                        out.print(", ") ;
-                }
-                else
-                    firstObject = false ;
-                int row1 = out.getRow() ;
-                writeNode(o) ;
-                int row2 = out.getRow();
-                lastObjectMultiLine = (row2 > row1) ;
-            }
-            out.decIndent(INDENT_OBJECT) ;
-        }
-
-        /** Write a predicate - jump to next line if deemed long */
-        private void writePredicate(Node p, int predicateMaxWidth, boolean first) {
-            if ( ! first ) {
-                print(" ;") ;
-                println() ;
-            }
-            int colPredicateStart = out.getAbsoluteIndent() ;
-
-            if ( printTypeKeyword && RDF_type.equals(p) )
-                print("a") ;
-            else
-                writeNode(p) ;
-
-            int colPredicateFinish = out.getCol() ;
-            int wPredicate = (colPredicateFinish - colPredicateStart) ;
-
-            if ( wPredicate > LONG_PREDICATE )
-                println() ;
-            else {
-                out.pad(predicateMaxWidth) ;
-                gap(GAP_P_O) ;
-            }
-        }
+        protected abstract void writePredicateObjectList(Collection<Triple> cluster) ;
 
-        private Map<Node, List<Node>> groupByPredicates(Collection<Triple> cluster) {
+        protected Map<Node, List<Node>> groupByPredicates(Collection<Triple> cluster) {
             SortedMap<Node, List<Node>> x = new TreeMap<>(compPredicates) ;
             for ( Triple t : cluster ) {
                 Node p = t.getPredicate() ;
@@ -807,7 +1153,7 @@ public abstract class TurtleShell {
         }
 
         // Compact if one triple, or one predicate and several non-pretty objects.
-        private boolean isCompact(Collection<Triple> cluster) {
+        protected boolean isCompact(Collection<Triple> cluster) {
             Node predicate = null;
             for ( Triple t : cluster ) {
                 Node p = t.getPredicate() ;
@@ -824,126 +1170,13 @@ public abstract class TurtleShell {
             return true;
         }
 
-        // [ :p "abc" ] .  or    [] : "abc" .
-        private void writeNestedObjectTopLevel(Node subject) {
-            if ( true ) {
-                writeNestedObject(subject) ;
-                out.println(" .") ;
-            } else {
-                // Alternative.
-                Collection<Triple> cluster = triplesOfSubject(subject) ;
-                print("[]") ;
-                writeClusterPredicateObjectList(0, cluster) ;
-            }
-        }
-
-        private void writeNestedObject(Node node) {
-            Collection<Triple> x = triplesOfSubject(node) ;
-
-            if ( x.isEmpty() ) {
-                print("[] ") ;
-                return ;
-            }
-
-            if ( isCompact(x) ) {
-                print("[ ") ;
-                out.incIndent(2) ;
-                writePredicateObjectList(x) ;
-                out.decIndent(2) ;
-                print(" ]") ;
-                return ;
-            }
-
-            int indent0 = out.getAbsoluteIndent() ;
-            int here = out.getCol() ;
-            out.setAbsoluteIndent(here) ;
-            print("[ ") ;
-            out.incIndent(2) ;
-            writePredicateObjectList(x) ;
-            out.decIndent(2) ;
-            if ( true ) {
-                println() ; // Newline for "]"
-                print("]") ;
-            } else { // Compact
-                print(" ]") ;
-            }
-            out.setAbsoluteIndent(indent0) ;
-        }
-
-        // Write a list
-        private void writeList(List<Node> elts) {
-            if ( elts.size() == 0 ) {
-                out.print("()") ;
-                return ;
-            }
-
-            if ( false ) {
-                out.print("(") ;
-                for ( Node n : elts ) {
-                    out.print(" ") ;
-                    writeNodePretty(n) ;
-                }
-                out.print(" )") ;
-            }
-
-            if ( true ) {
-                // "fresh line mode" means printed one on new line
-                // Multi line items are ones that can be multiple lines. Non-literals.
-                // Was the previous row a multiLine?
-                boolean lastItemFreshLine = false ;
-                // Have there been any items that causes "fresh line" mode?
-                boolean multiLineAny = false ;
-                boolean first = true ;
-
-                // Where we started.
-                int originalIndent = out.getAbsoluteIndent() ;
-                // Rebase indent here.
-                int x = out.getCol() ;
-                out.setAbsoluteIndent(x);
-
-                out.print("(") ;
-                out.incIndent(2);
-                for ( Node n : elts ) {
-
-                    // Print this item on a fresh line? (still to check: first line)
-                    boolean thisItemFreshLine = /* multiLineAny | */ n.isBlank() ;
-
-                    // Special case List in List.
-                    // Start on this line if last item was on this line.
-                    if ( lists.containsKey(n) )
-                        thisItemFreshLine = lastItemFreshLine ;
-
-                    // Starting point.
-                    if ( ! first ) {
-                        if ( lastItemFreshLine | thisItemFreshLine )
-                            out.println() ;
-                        else
-                            out.print(" ") ;
-                    }
+        protected abstract void writeNestedObjectTopLevel(Node subject) ;
 
-                    first = false ;
-                    //Literals with newlines: int x1 = out.getRow() ;
-                    // Adds INDENT_OBJECT even for a [ one triple ]
-                    // Special case [ one triple ]??
-                    writeNodePretty(n) ;
-                    //Literals with newlines:int x2 = out.getRow() ;
-                    //Literals with newlines: boolean multiLineAnyway = ( x1 != x2 ) ;
-                    lastItemFreshLine = thisItemFreshLine ;
-                    multiLineAny  = multiLineAny | thisItemFreshLine ;
+        protected abstract void writeNestedObject(Node node) ;
 
-                }
-                if ( multiLineAny )
-                    out.println() ;
-                else
-                    out.print(" ") ;
-                out.decIndent(2);
-                out.setAbsoluteIndent(x);
-                out.print(")") ;
-                out.setAbsoluteIndent(originalIndent) ;
-            }
-        }
+        protected abstract void writeList(List<Node> elts) ;
 
-        private boolean isPrettyNode(Node n) {
+        protected boolean isPrettyNode(Node n) {
             // Maybe ought to be the same test as writePredicateObjectList
             // Order matters? - one connected objects may include list elements.
             if ( allowDeepPretty ) {
@@ -958,7 +1191,7 @@ public abstract class TurtleShell {
         }
 
         // --> write S or O??
-        private void writeNodePretty(Node obj) {
+        protected void writeNodePretty(Node obj) {
             // Assumes "isPrettyNode" is true.
             // Order matters? - one connected objects may include list elements.
             if ( lists.containsKey(obj) )
@@ -980,12 +1213,8 @@ public abstract class TurtleShell {
         // Other.
         // Sorted by URI.
 
-        private void write_S_P_Gap() {
-            if ( out.getCol() > LONG_SUBJECT )
-                out.println() ;
-            else
-                gap(GAP_S_P) ;
-        }
+        protected abstract void write_S_P_Gap() ;
+
     }
 
     // Order of properties.
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/writer/TurtleWriterLong.java b/jena-arq/src/main/java/org/apache/jena/riot/writer/TurtleWriterLong.java
new file mode 100644
index 0000000000..086bddf566
--- /dev/null
+++ b/jena-arq/src/main/java/org/apache/jena/riot/writer/TurtleWriterLong.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.riot.writer ;
+
+import org.apache.jena.atlas.io.IndentedWriter ;
+import org.apache.jena.graph.Graph ;
+import org.apache.jena.riot.RIOT;
+import org.apache.jena.riot.system.PrefixMap ;
+import org.apache.jena.sparql.util.Context ;
+
+public class TurtleWriterLong extends TurtleWriter {
+
+    @Override
+    protected void output(IndentedWriter iOut, Graph graph, PrefixMap prefixMap, String baseURI, Context context) {
+        context.set(RIOT.symTurtleIndentStyle, "long");
+        super.output(iOut, graph, prefixMap, baseURI, context);
+    }
+}
diff --git a/jena-arq/src/main/java/org/apache/jena/riot/writer/WriterLib.java b/jena-arq/src/main/java/org/apache/jena/riot/writer/WriterLib.java
index 3adc22be85..0132a39e58 100644
--- a/jena-arq/src/main/java/org/apache/jena/riot/writer/WriterLib.java
+++ b/jena-arq/src/main/java/org/apache/jena/riot/writer/WriterLib.java
@@ -26,6 +26,8 @@ import org.apache.jena.sparql.util.Context;
 
     private static final DirectiveStyle dftDirectiveStyle = DirectiveStyle.AT;
 
+    private static final IndentStyle dftIndentStyle = IndentStyle.WIDE;
+
     // Determine the directive style (applies to PREFIX and BASE).
     /*package*/ static DirectiveStyle directiveStyle(Context context) {
         if ( context == null )
@@ -43,4 +45,19 @@ import org.apache.jena.sparql.util.Context;
         // Default choice; includes null in context.
         return dftDirectiveStyle;
     }
+
+    /*package*/ static IndentStyle indentStyle(Context context) {
+        if ( context == null )
+            return dftIndentStyle;
+        Object x = context.get(RIOT.symTurtleIndentStyle) ;
+
+        if ( x instanceof String ) {
+            String s = (String)x ;
+            IndentStyle style = IndentStyle.create(s);
+            return style == null ? dftIndentStyle : style;
+        }
+
+        // Default choice; includes null in context.
+        return dftIndentStyle;
+    }
 }
diff --git a/jena-arq/src/test/java/org/apache/jena/riot/TestTurtleWriter.java b/jena-arq/src/test/java/org/apache/jena/riot/TestTurtleWriter.java
index 5d9ba405c1..30559d2e08 100644
--- a/jena-arq/src/test/java/org/apache/jena/riot/TestTurtleWriter.java
+++ b/jena-arq/src/test/java/org/apache/jena/riot/TestTurtleWriter.java
@@ -40,10 +40,12 @@ public class TestTurtleWriter {
         x.add(new Object[]{"Turtle/Pretty", RDFFormat.TURTLE_PRETTY});
         x.add(new Object[]{"Turtle/Blocks", RDFFormat.TURTLE_BLOCKS});
         x.add(new Object[]{"Turtle/Flat", RDFFormat.TURTLE_FLAT});
+        x.add(new Object[]{"Turtle/Long", RDFFormat.TURTLE_LONG});
         x.add(new Object[]{"Trig", RDFFormat.TRIG});
         x.add(new Object[]{"Trig/Pretty", RDFFormat.TRIG_PRETTY});
         x.add(new Object[]{"Trig/Blocks", RDFFormat.TRIG_BLOCKS});
         x.add(new Object[]{"Trig/Flat", RDFFormat.TRIG_FLAT});
+        x.add(new Object[]{"Trig/Long", RDFFormat.TRIG_LONG});
         return x ; 
     }
     
diff --git a/jena-arq/src/test/java/org/apache/jena/riot/TestTurtleWriterPretty.java b/jena-arq/src/test/java/org/apache/jena/riot/TestTurtleWriterPretty.java
index 8afce9f190..237f8f78ea 100644
--- a/jena-arq/src/test/java/org/apache/jena/riot/TestTurtleWriterPretty.java
+++ b/jena-arq/src/test/java/org/apache/jena/riot/TestTurtleWriterPretty.java
@@ -39,7 +39,9 @@ public class TestTurtleWriterPretty {
     public static Iterable<Object[]> data() {
         List<Object[]> x = new ArrayList<>() ;
         x.add(new Object[]{"Turtle/Pretty", RDFFormat.TURTLE_PRETTY});
+        x.add(new Object[]{"Turtle/Long", RDFFormat.TURTLE_LONG});
         x.add(new Object[]{"Trig/Pretty", RDFFormat.TRIG_PRETTY});
+        x.add(new Object[]{"Trig/Long", RDFFormat.TRIG_LONG});
         return x ; 
     }
     
diff --git a/jena-arq/src/test/java/org/apache/jena/riot/writer/TestRiotWriterGraph.java b/jena-arq/src/test/java/org/apache/jena/riot/writer/TestRiotWriterGraph.java
index 954c905c2a..84f1159274 100644
--- a/jena-arq/src/test/java/org/apache/jena/riot/writer/TestRiotWriterGraph.java
+++ b/jena-arq/src/test/java/org/apache/jena/riot/writer/TestRiotWriterGraph.java
@@ -51,6 +51,7 @@ public class TestRiotWriterGraph extends AbstractWriterTest
             , { RDFFormat.TURTLE_PRETTY }
             , { RDFFormat.TURTLE_BLOCKS }
             , { RDFFormat.TURTLE_FLAT }
+            , { RDFFormat.TURTLE_LONG }
             , { RDFFormat.RDFXML }
             , { RDFFormat.RDFXML_PRETTY }
             , { RDFFormat.RDFXML_PLAIN }
@@ -73,6 +74,7 @@ public class TestRiotWriterGraph extends AbstractWriterTest
             , { RDFFormat.TRIG_PRETTY }
             , { RDFFormat.TRIG_BLOCKS }
             , { RDFFormat.TRIG_FLAT }
+            , { RDFFormat.TRIG_LONG }
             , { RDFFormat.NQUADS_UTF8}
             , { RDFFormat.NQUADS_ASCII}
             , { RDFFormat.NQUADS}
diff --git a/jena-arq/src/test/java/org/apache/jena/riot/writer/TestTurtleWriter.java b/jena-arq/src/test/java/org/apache/jena/riot/writer/TestTurtleWriter.java
index 8f729a5721..bae94fa6e2 100644
--- a/jena-arq/src/test/java/org/apache/jena/riot/writer/TestTurtleWriter.java
+++ b/jena-arq/src/test/java/org/apache/jena/riot/writer/TestTurtleWriter.java
@@ -81,9 +81,6 @@ public class TestTurtleWriter {
     @Test
     public void bnode_cycles_02() { blankNodeLang(cycle1, RDFFormat.TURTLE_BLOCKS) ; }
 
-    @Test
-    public void bnode_cycles_03() { blankNodeLang(cycle1, RDFFormat.TURTLE_FLAT) ; }
-
     @Test
     public void bnode_cycles_04() { blankNodeLang(cycle1, RDFFormat.TURTLE_PRETTY) ; }
 
@@ -99,6 +96,9 @@ public class TestTurtleWriter {
     @Test
     public void bnode_cycles_08() { blankNodeLang(cycle2, RDFFormat.TURTLE_PRETTY) ; }
 
+    @Test
+    public void bnode_cycles_09() { blankNodeLang(cycle2, RDFFormat.TURTLE_LONG) ; }
+
     @Test
     public void bnode_cycles() {
         Model m = RDFDataMgr.loadModel("testing/DAWG-Final/construct/data-ident.ttl");