You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@marmotta.apache.org by an...@apache.org on 2013/09/27 03:18:14 UTC

git commit: Remove hardcoded formats in SparqlServiceImpl and add service factories

Updated Branches:
  refs/heads/develop 9125265af -> 10d862669


Remove hardcoded formats in SparqlServiceImpl and add service factories

This makes it extensible at runtime using any and all available formats
and mime types. The special requirement for runtime binding of a
TemplatingService in the SPARQLResultsHTMLWriter is accommodated for
using a setting that is checked when the query is about to be processed.

Project: http://git-wip-us.apache.org/repos/asf/incubator-marmotta/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-marmotta/commit/10d86266
Tree: http://git-wip-us.apache.org/repos/asf/incubator-marmotta/tree/10d86266
Diff: http://git-wip-us.apache.org/repos/asf/incubator-marmotta/diff/10d86266

Branch: refs/heads/develop
Commit: 10d86266930c2695993be29aefd0109f31d25b5c
Parents: 9125265
Author: Peter Ansell <p_...@yahoo.com>
Authored: Fri Sep 27 11:16:09 2013 +1000
Committer: Peter Ansell <p_...@yahoo.com>
Committed: Fri Sep 27 11:16:09 2013 +1000

----------------------------------------------------------------------
 .../services/sparql/SparqlServiceImpl.java      |  59 ++--
 .../sparqlhtml/SPARQLBooleanHTMLFormat.java     |  32 ++
 .../sparqlhtml/SPARQLBooleanHTMLWriter.java     | 293 ++++++++++---------
 .../SPARQLBooleanHTMLWriterFactory.java         |  49 ++++
 .../sparqlio/sparqlhtml/SPARQLHTMLSettings.java |  38 +++
 .../sparqlhtml/SPARQLResultsHTMLFormat.java     |  32 ++
 .../sparqlhtml/SPARQLResultsHTMLWriter.java     |  34 ++-
 .../SPARQLResultsHTMLWriterFactory.java         |  49 ++++
 ...ery.resultio.BooleanQueryResultWriterFactory |   1 +
 ...query.resultio.TupleQueryResultWriterFactory |   2 +
 10 files changed, 404 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparql/SparqlServiceImpl.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparql/SparqlServiceImpl.java b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparql/SparqlServiceImpl.java
index f123cfa..f034373 100644
--- a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparql/SparqlServiceImpl.java
+++ b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparql/SparqlServiceImpl.java
@@ -44,8 +44,8 @@ import org.apache.marmotta.platform.core.exception.MarmottaException;
 import org.apache.marmotta.platform.sparql.api.sparql.QueryType;
 import org.apache.marmotta.platform.sparql.api.sparql.SparqlService;
 import org.apache.marmotta.platform.sparql.services.sparqlio.rdf.SPARQLGraphResultWriter;
-import org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml.SPARQLBooleanHTMLWriter;
-import org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml.SPARQLResultsHTMLWriter;
+import org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml.SPARQLResultsHTMLFormat;
+import org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml.SPARQLHTMLSettings;
 import org.apache.marmotta.platform.sparql.webservices.SparqlWebService;
 import org.openrdf.model.Value;
 import org.openrdf.query.Binding;
@@ -56,6 +56,7 @@ import org.openrdf.query.MalformedQueryException;
 import org.openrdf.query.Query;
 import org.openrdf.query.QueryEvaluationException;
 import org.openrdf.query.QueryLanguage;
+import org.openrdf.query.QueryResultHandlerException;
 import org.openrdf.query.TupleQuery;
 import org.openrdf.query.TupleQueryResult;
 import org.openrdf.query.TupleQueryResultHandlerException;
@@ -352,8 +353,8 @@ public class SparqlServiceImpl implements SparqlService {
 
     private void query(BooleanQuery query, BooleanQueryResultWriter writer) throws QueryEvaluationException {
         try {
-            writer.write(query.evaluate());
-        } catch (IOException e) {
+            writer.handleBoolean(query.evaluate());
+        } catch (QueryResultHandlerException e) {
             throw new QueryEvaluationException("error while writing query boolean result: ",e);
         }
     }
@@ -497,41 +498,37 @@ public class SparqlServiceImpl implements SparqlService {
     }
 
     private TupleQueryResultWriter getTupleResultWriter(String format, OutputStream os) {
-        //build outputwriter
-        final TupleQueryResultWriter out;
+        TupleQueryResultFormat resultFormat;
         if(format == null) {
-            out = QueryResultIO.createWriter(TupleQueryResultFormat.SPARQL, os);
-        } else if(SparqlWritersHelper.parseSubType(format).equals("html")) {
-            out = new SPARQLResultsHTMLWriter(os, templatingService);
-        } else if(SparqlWritersHelper.parseSubType(format).equals("json")) {
-            out = QueryResultIO.createWriter(TupleQueryResultFormat.JSON, os);
-        } else if(SparqlWritersHelper.parseSubType(format).equals("xml")) {
-            out = QueryResultIO.createWriter(TupleQueryResultFormat.SPARQL, os);
-        } else if(SparqlWritersHelper.parseSubType(format).equals("csv")) {
-            out = QueryResultIO.createWriter(TupleQueryResultFormat.CSV, os);
-        } else throw new InvalidArgumentException("could not produce format "+format);
-        return out;
+            resultFormat = TupleQueryResultFormat.SPARQL;
+        } else {
+            resultFormat = QueryResultIO.getWriterFormatForMIMEType(format);
+            if(resultFormat == null) {
+                throw new InvalidArgumentException("could not produce format "+format);
+            }
+        } 
+        TupleQueryResultWriter writer = QueryResultIO.createWriter(resultFormat, os);
+        if(resultFormat.equals(SPARQLResultsHTMLFormat.SPARQL_RESULTS_HTML)) {
+            writer.getWriterConfig().set(SPARQLHTMLSettings.TEMPLATING_SERVICE, templatingService);
+        }
+        return writer;
     }
 
     private BooleanQueryResultWriter getBooleanResultWriter(String format, OutputStream os) {
-        //build outputwriter
-        final BooleanQueryResultWriter out;
+        BooleanQueryResultFormat resultFormat;
         if(format == null) {
-            out = QueryResultIO.createWriter(BooleanQueryResultFormat.SPARQL, os);
-        } else if(SparqlWritersHelper.parseSubType(format).equals("html")) {
-            out = new SPARQLBooleanHTMLWriter(os);
-        } else if(SparqlWritersHelper.parseSubType(format).equals("json")) {
-            out = QueryResultIO.createWriter(BooleanQueryResultFormat.JSON, os);
-        } else if(SparqlWritersHelper.parseSubType(format).equals("xml")) {
-            out = QueryResultIO.createWriter(BooleanQueryResultFormat.SPARQL, os);
-        } else if(SparqlWritersHelper.parseSubType(format).equals("csv")) {
-            out = QueryResultIO.createWriter(BooleanQueryResultFormat.TEXT, os);
-        } else throw new InvalidArgumentException("could not produce format "+format);
-        return out;
+            resultFormat = BooleanQueryResultFormat.SPARQL;
+        } else {
+            resultFormat = QueryResultIO.getBooleanWriterFormatForMIMEType(format);
+            if(resultFormat == null) {
+                throw new InvalidArgumentException("could not produce format "+format);
+            }
+        } 
+        return QueryResultIO.createWriter(resultFormat, os);
     }
 
     private SPARQLGraphResultWriter getGraphResultWriter(String format, OutputStream os) {
-        return new SPARQLGraphResultWriter(os,format);
+        return new SPARQLGraphResultWriter(os, format);
     }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLFormat.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLFormat.java b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLFormat.java
new file mode 100644
index 0000000..73b22c0
--- /dev/null
+++ b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLFormat.java
@@ -0,0 +1,32 @@
+/**
+ * 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.marmotta.platform.sparql.services.sparqlio.sparqlhtml;
+
+import java.nio.charset.Charset;
+
+import org.openrdf.query.resultio.BooleanQueryResultFormat;
+
+/**
+ * Results format for Marmotta HTML Rendering of SPARQL Boolean Results.
+ * 
+ * @author Peter Ansell
+ */
+public final class SPARQLBooleanHTMLFormat
+{
+    public static final BooleanQueryResultFormat SPARQL_BOOLEAN_HTML = new BooleanQueryResultFormat("HTML", "text/html", Charset.forName("utf-8"), "html");
+}

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriter.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriter.java b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriter.java
index 3a94b2d..acdb561 100644
--- a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriter.java
+++ b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriter.java
@@ -1,19 +1,16 @@
 /**
- * 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.
+ * 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.marmotta.platform.sparql.services.sparqlio.sparqlhtml;
 
@@ -47,7 +44,7 @@ import org.openrdf.query.TupleQueryResultHandlerException;
 import org.openrdf.query.resultio.BooleanQueryResultFormat;
 import org.openrdf.query.resultio.BooleanQueryResultWriter;
 import org.openrdf.query.resultio.QueryResultFormat;
-import org.openrdf.query.resultio.sparqlxml.SPARQLBooleanXMLWriter;
+import org.openrdf.query.resultio.QueryResultIO;
 import org.openrdf.rio.RioSetting;
 import org.openrdf.rio.WriterConfig;
 import org.slf4j.Logger;
@@ -59,155 +56,175 @@ import org.slf4j.LoggerFactory;
  * @author Sebastian Schaffert
  * @author Sergio Fernández
  */
-public class SPARQLBooleanHTMLWriter implements BooleanQueryResultWriter {
-
+public class SPARQLBooleanHTMLWriter implements BooleanQueryResultWriter
+{
+    
     private static final Logger log = LoggerFactory.getLogger(SPARQLBooleanHTMLWriter.class);
-
+    
     private OutputStream out;
-    private ByteArrayOutputStream xmlOut;
-
-    private SPARQLBooleanXMLWriter writer;
+    
     private WriterConfig config;
-
-    private Templates stylesheet;
-
-    public SPARQLBooleanHTMLWriter(OutputStream out) {
+    
+    public SPARQLBooleanHTMLWriter(OutputStream out)
+    {
         this.out = out;
-        this.xmlOut = new ByteArrayOutputStream();
-        this.writer = new SPARQLBooleanXMLWriter(xmlOut);
-
-        Source s_stylesheet = new StreamSource(SPARQLBooleanHTMLWriter.class.getResourceAsStream("style.xsl"));
-        try {
-            stylesheet = TransformerFactory.newInstance().newTemplates(s_stylesheet);
-        } catch (TransformerConfigurationException e) {
-            log.error("could not compile stylesheet for rendering SPARQL results; result display not available!");
-        }
     }
-
+    
     /**
      * Gets the query result format that this writer uses.
      */
     @Override
-    public BooleanQueryResultFormat getBooleanQueryResultFormat() {
-        return new BooleanQueryResultFormat("SPARQL/HTML","text/html", Charset.forName("UTF-8"), "html");
+    public BooleanQueryResultFormat getBooleanQueryResultFormat()
+    {
+        return SPARQLBooleanHTMLFormat.SPARQL_BOOLEAN_HTML;
     }
-
+    
     /**
      * Writes the specified boolean value.
      */
     @Override
     @Deprecated
-    public void write(boolean value) throws IOException {
-    	try {
-			handleBoolean(value);
-		} catch (QueryResultHandlerException e) {
-			throw new IOException(e);
-		}
+    public void write(boolean value) throws IOException
+    {
+        try
+        {
+            handleBoolean(value);
+        }
+        catch(QueryResultHandlerException e)
+        {
+            throw new IOException(e);
+        }
     }
-
-	@Override
-	public void endHeader() throws QueryResultHandlerException {
-		
-	}
-
-	@Override
-	public QueryResultFormat getQueryResultFormat() {
-		return new QueryResultFormat("HTML", "text/html", Charset.forName("utf-8"), "html");
-	}
-
-	@Override
-	public Collection<RioSetting<?>> getSupportedSettings() {
-		return new ArrayList<RioSetting<?>>();
-	}
-
-	@Override
-	public WriterConfig getWriterConfig() {
-		return config;
-	}
-
-	@Override
-	public void handleNamespace(String arg0, String arg1) throws QueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void handleStylesheet(String arg0) throws QueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void setWriterConfig(WriterConfig config) {
-		this.config = config;
-	}
-
-	@Override
-	public void startDocument() throws QueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void startHeader() throws QueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void endQueryResult() throws TupleQueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void handleBoolean(boolean value) throws QueryResultHandlerException { 	
-        try {
-            writer.write(value);
-
+    
+    @Override
+    public void handleBoolean(boolean value) throws QueryResultHandlerException
+    {
+        try
+        {
+            // Create a SPARQL/XML representation that will be transformed to HTML using a stylesheet
+            ByteArrayOutputStream xmlOut = new ByteArrayOutputStream();
+            QueryResultIO.writeBoolean(value, BooleanQueryResultFormat.SPARQL, xmlOut);
             byte[] queryResult = xmlOut.toByteArray();
-
+            
             // get server uri
             String server_uri = CDIContext.getInstance(ConfigurationService.class).getServerUri();
-
+            
             BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out));
-        	
-        	Source input = new StreamSource(new ByteArrayInputStream(queryResult));
-
+            
+            Source input = new StreamSource(new ByteArrayInputStream(queryResult));
+            
+            Source s_stylesheet = new StreamSource(SPARQLBooleanHTMLWriter.class.getResourceAsStream("style.xsl"));
+            Templates stylesheet = TransformerFactory.newInstance().newTemplates(s_stylesheet);
             Transformer transformer = stylesheet.newTransformer();
             transformer.setParameter("serverurl", server_uri);
-
+            
             JDOMResult result = new JDOMResult();
-            transformer.transform(input,result);
+            transformer.transform(input, result);
             Document output = result.getDocument();
-
+            
             XMLOutputter printer = new XMLOutputter(Format.getPrettyFormat());
             printer.output(output, writer);
             writer.flush();
-        } catch (Exception ex) {
-            throw new QueryResultHandlerException("error while transforming XML results to HTML",ex);
-        } finally {
-            //writer.close();
         }
-		
-	}
-
-	@Override
-	public void handleLinks(List<String> links) throws QueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void handleSolution(BindingSet bindings) throws TupleQueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
-	public void startQueryResult(List<String> start) throws TupleQueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
+        catch(TransformerConfigurationException e)
+        {
+            log.error("could not compile stylesheet for rendering SPARQL results; result display not available!");
+            throw new QueryResultHandlerException("could not compile stylesheet for rendering SPARQL results; result display not available!", e);
+        }
+        catch(Exception ex)
+        {
+            throw new QueryResultHandlerException("error while transforming XML results to HTML", ex);
+        }
+        finally
+        {
+            // writer.close();
+        }
+    }
+    
+    @Override
+    public QueryResultFormat getQueryResultFormat()
+    {
+        return getBooleanQueryResultFormat();
+    }
+    
+    @Override
+    public WriterConfig getWriterConfig()
+    {
+        return config;
+    }
+    
+    @Override
+    public void setWriterConfig(WriterConfig config)
+    {
+        this.config = config;
+    }
+    
+    @Override
+    public Collection<RioSetting<?>> getSupportedSettings()
+    {
+        return new ArrayList<RioSetting<?>>();
+    }
+    
+    @Override
+    public void endHeader() throws QueryResultHandlerException
+    {
+        
+    }
+    
+    @Override
+    public void handleNamespace(String arg0, String arg1) throws QueryResultHandlerException
+    {
+        // TODO Auto-generated method stub
+        
+    }
+    
+    @Override
+    public void handleStylesheet(String arg0) throws QueryResultHandlerException
+    {
+        // TODO Auto-generated method stub
+        
+    }
+    
+    @Override
+    public void startDocument() throws QueryResultHandlerException
+    {
+        // TODO Auto-generated method stub
+        
+    }
+    
+    @Override
+    public void startHeader() throws QueryResultHandlerException
+    {
+        // TODO Auto-generated method stub
+        
+    }
+    
+    @Override
+    public void handleLinks(List<String> linkUrls) throws QueryResultHandlerException
+    {
+        // TODO Auto-generated method stub
+        
+    }
+    
+    @Override
+    public void startQueryResult(List<String> bindingNames) throws TupleQueryResultHandlerException
+    {
+        // TODO Auto-generated method stub
+        
+    }
+    
+    @Override
+    public void endQueryResult() throws TupleQueryResultHandlerException
+    {
+        // TODO Auto-generated method stub
+        
+    }
+    
+    @Override
+    public void handleSolution(BindingSet bindingSet) throws TupleQueryResultHandlerException
+    {
+        // TODO Auto-generated method stub
+        
+    }
     
 }

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriterFactory.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriterFactory.java b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriterFactory.java
new file mode 100644
index 0000000..43a9947
--- /dev/null
+++ b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLBooleanHTMLWriterFactory.java
@@ -0,0 +1,49 @@
+/**
+ * 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.marmotta.platform.sparql.services.sparqlio.sparqlhtml;
+
+import java.io.OutputStream;
+
+import org.openrdf.query.resultio.BooleanQueryResultFormat;
+import org.openrdf.query.resultio.BooleanQueryResultWriter;
+import org.openrdf.query.resultio.BooleanQueryResultWriterFactory;
+
+/**
+ * A {@link BooleanQueryResultWriterFactory} for writers of SPARQL HTML query
+ * boolean results.
+ * 
+ * @author Peter Ansell
+ */
+public class SPARQLBooleanHTMLWriterFactory implements BooleanQueryResultWriterFactory {
+
+	/**
+	 * Returns {@link SPARQLBooleanHTMLFormat#SPARQL_BOOLEAN_HTML}.
+	 */
+    @Override
+	public BooleanQueryResultFormat getBooleanQueryResultFormat() {
+		return SPARQLBooleanHTMLFormat.SPARQL_BOOLEAN_HTML;
+	}
+
+	/**
+	 * Returns a new instance of SPARQLBooleanHTMLWriter.
+	 */
+    @Override
+	public BooleanQueryResultWriter getWriter(OutputStream out) {
+		return new SPARQLBooleanHTMLWriter(out);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLHTMLSettings.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLHTMLSettings.java b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLHTMLSettings.java
new file mode 100644
index 0000000..2db9ca0
--- /dev/null
+++ b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLHTMLSettings.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml;
+
+import org.apache.marmotta.platform.core.api.templating.TemplatingService;
+import org.openrdf.rio.RioSetting;
+import org.openrdf.rio.helpers.RioSettingImpl;
+
+/**
+ * Contains settings relevant to SPARQL HTML Query Results Writers.
+ * 
+ * @author Peter Ansell
+ */
+public final class SPARQLHTMLSettings
+{
+    /**
+     * The {@link TemplatingService} used by the SPARQL Results HTML Writer.
+     * <p>
+     * Defaults to null
+     */
+    public static final RioSetting<TemplatingService> TEMPLATING_SERVICE = new RioSettingImpl<TemplatingService>(
+    "org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml.templatingservice", "Templating service for SPARQL Results HTML Writer", null);
+}

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLFormat.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLFormat.java b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLFormat.java
new file mode 100644
index 0000000..381c18e
--- /dev/null
+++ b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLFormat.java
@@ -0,0 +1,32 @@
+/**
+ * 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.marmotta.platform.sparql.services.sparqlio.sparqlhtml;
+
+import java.nio.charset.Charset;
+
+import org.openrdf.query.resultio.TupleQueryResultFormat;
+
+/**
+ * Results format for Marmotta HTML Rendering of SPARQL Tuple Results.
+ * 
+ * @author Peter Ansell
+ */
+public final class SPARQLResultsHTMLFormat
+{
+    public static final TupleQueryResultFormat SPARQL_RESULTS_HTML = new TupleQueryResultFormat("SPARQL/HTML", "text/html", Charset.forName("UTF-8"), "html");
+}

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriter.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriter.java b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriter.java
index ffafd4c..9cf348e 100644
--- a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriter.java
+++ b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriter.java
@@ -19,7 +19,6 @@ package org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml;
 
 import java.io.OutputStream;
 import java.io.OutputStreamWriter;
-import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashMap;
@@ -29,6 +28,7 @@ import java.util.Map;
 import org.apache.marmotta.platform.core.api.templating.TemplatingService;
 import org.openrdf.query.BindingSet;
 import org.openrdf.query.QueryResultHandlerException;
+import org.openrdf.query.TupleQueryResultHandlerBase;
 import org.openrdf.query.TupleQueryResultHandlerException;
 import org.openrdf.query.resultio.QueryResultFormat;
 import org.openrdf.query.resultio.TupleQueryResultFormat;
@@ -43,7 +43,7 @@ import org.slf4j.LoggerFactory;
  * 
  * @author Sergio Fernández
  */
-public class SPARQLResultsHTMLWriter implements TupleQueryResultWriter {
+public class SPARQLResultsHTMLWriter extends TupleQueryResultHandlerBase implements TupleQueryResultWriter {
 
 	private static final Logger log = LoggerFactory.getLogger(SPARQLResultsHTMLWriter.class);
     
@@ -52,7 +52,7 @@ public class SPARQLResultsHTMLWriter implements TupleQueryResultWriter {
 	private static final String RESULT_TEMPLATE = "sparql_select_result.ftl";
 
 	private static final String END_TEMPLATE = "sparql_select_end.ftl";
-	
+
     private OutputStream out;
     
     private List<String> vars;
@@ -61,18 +61,28 @@ public class SPARQLResultsHTMLWriter implements TupleQueryResultWriter {
     
     private WriterConfig config;
     
-    public SPARQLResultsHTMLWriter(OutputStream out, TemplatingService templatingService) {
+    public SPARQLResultsHTMLWriter(OutputStream out) {
         this.out = out;
+    }
+    
+    public SPARQLResultsHTMLWriter(OutputStream out, TemplatingService templatingService) {
+        this(out);
         this.templatingService = templatingService;
     }
-
-	@Override
+    
+    @Override
 	public TupleQueryResultFormat getTupleQueryResultFormat() {
-		return new TupleQueryResultFormat("SPARQL/HTML", "text/html", Charset.forName("UTF-8"), "html");
+		return SPARQLResultsHTMLFormat.SPARQL_RESULTS_HTML;
 	}
 
 	@Override
 	public void startQueryResult(List<String> vars) throws TupleQueryResultHandlerException {
+	    if(templatingService == null) {
+	        templatingService = getWriterConfig().get(SPARQLHTMLSettings.TEMPLATING_SERVICE);
+	        if(templatingService == null) {
+	            throw new IllegalStateException("Templating service was not setup");
+	        }
+	    }
         Map<String, Object> data = new HashMap<String, Object>();
         data.put("vars", vars);
         this.vars = vars;
@@ -117,22 +127,14 @@ public class SPARQLResultsHTMLWriter implements TupleQueryResultWriter {
 	}
 
 	@Override
-	public void handleBoolean(boolean arg0) throws QueryResultHandlerException {
-		// TODO Auto-generated method stub
-		
-	}
-
-	@Override
 	public void handleLinks(List<String> arg0)
 			throws QueryResultHandlerException {
 		// TODO Auto-generated method stub
-		
 	}
 
 	@Override
 	public QueryResultFormat getQueryResultFormat() {
-		// TODO Auto-generated method stub
-		return null;
+	    return getTupleQueryResultFormat();
 	}
 
 	@Override

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriterFactory.java
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriterFactory.java b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriterFactory.java
new file mode 100644
index 0000000..c94d3c3
--- /dev/null
+++ b/platform/marmotta-sparql/src/main/java/org/apache/marmotta/platform/sparql/services/sparqlio/sparqlhtml/SPARQLResultsHTMLWriterFactory.java
@@ -0,0 +1,49 @@
+/**
+ * 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.marmotta.platform.sparql.services.sparqlio.sparqlhtml;
+
+import java.io.OutputStream;
+
+import org.openrdf.query.resultio.TupleQueryResultFormat;
+import org.openrdf.query.resultio.TupleQueryResultWriter;
+import org.openrdf.query.resultio.TupleQueryResultWriterFactory;
+
+/**
+ * A {@link TupleQueryResultWriterFactory} for writers of SPARQL HTML query
+ * tuple results.
+ * 
+ * @author Peter Ansell
+ */
+public class SPARQLResultsHTMLWriterFactory implements TupleQueryResultWriterFactory {
+
+	/**
+	 * Returns {@link SPARQLResultsHTMLFormat#SPARQL_RESULTS_HTML}.
+	 */
+    @Override
+	public TupleQueryResultFormat getTupleQueryResultFormat() {
+		return SPARQLResultsHTMLFormat.SPARQL_RESULTS_HTML;
+	}
+
+	/**
+	 * Returns a new instance of SPARQLResultsHTMLWriter.
+	 */
+	@Override
+	public TupleQueryResultWriter getWriter(OutputStream out) {
+		return new SPARQLResultsHTMLWriter(out);
+	}
+}

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/resources/META-INF/services/org.openrdf.query.resultio.BooleanQueryResultWriterFactory
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/resources/META-INF/services/org.openrdf.query.resultio.BooleanQueryResultWriterFactory b/platform/marmotta-sparql/src/main/resources/META-INF/services/org.openrdf.query.resultio.BooleanQueryResultWriterFactory
new file mode 100644
index 0000000..fe55ac1
--- /dev/null
+++ b/platform/marmotta-sparql/src/main/resources/META-INF/services/org.openrdf.query.resultio.BooleanQueryResultWriterFactory
@@ -0,0 +1 @@
+org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml.SPARQLBooleanHTMLWriterFactory

http://git-wip-us.apache.org/repos/asf/incubator-marmotta/blob/10d86266/platform/marmotta-sparql/src/main/resources/META-INF/services/org.openrdf.query.resultio.TupleQueryResultWriterFactory
----------------------------------------------------------------------
diff --git a/platform/marmotta-sparql/src/main/resources/META-INF/services/org.openrdf.query.resultio.TupleQueryResultWriterFactory b/platform/marmotta-sparql/src/main/resources/META-INF/services/org.openrdf.query.resultio.TupleQueryResultWriterFactory
new file mode 100644
index 0000000..c9eff25
--- /dev/null
+++ b/platform/marmotta-sparql/src/main/resources/META-INF/services/org.openrdf.query.resultio.TupleQueryResultWriterFactory
@@ -0,0 +1,2 @@
+org.apache.marmotta.platform.sparql.services.sparqlio.sparqlhtml.SPARQLResultsHTMLWriterFactory
+