You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@jena.apache.org by an...@apache.org on 2015/03/07 17:41:36 UTC

[1/4] jena git commit: JENA-895 : Do backup inside a read transaction.

Repository: jena
Updated Branches:
  refs/heads/master 51794850e -> 6d17e32dc


JENA-895 : Do backup inside a read transaction.

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

Branch: refs/heads/master
Commit: 3ecbdba6c11c1ae1ce197e937a3cefcf871e3a96
Parents: 5179485
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Mar 7 16:32:28 2015 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Mar 7 16:32:28 2015 +0000

----------------------------------------------------------------------
 .../java/org/apache/jena/fuseki/mgt/Backup.java | 25 ++++++++++++++++++--
 1 file changed, 23 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/3ecbdba6/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/Backup.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/Backup.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/Backup.java
index 66aa4c4..b0a94e1 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/Backup.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/Backup.java
@@ -31,7 +31,10 @@ import org.apache.jena.fuseki.server.FusekiServer ;
 import org.apache.jena.riot.Lang ;
 import org.apache.jena.riot.RDFDataMgr ;
 
+import com.hp.hpl.jena.query.ReadWrite ;
 import com.hp.hpl.jena.sparql.core.DatasetGraph ;
+import com.hp.hpl.jena.sparql.core.Transactional ;
+import com.hp.hpl.jena.sparql.core.TransactionalNull ;
 import com.hp.hpl.jena.sparql.util.Utils ;
 
 /** Perform a backup */ 
@@ -58,7 +61,26 @@ public class Backup
     // same dataset multiple times at the same time. 
     private static Set<DatasetGraph> activeBackups = new HashSet<>() ;
     
-    public static void backup(DatasetGraph dsg, String backupfile) {
+    /** Perform a backup.
+     *  A backup is a dump of the datset in comrpessed N-Quads, done inside a transaction.
+     */
+    public static void backup(Transactional transactional, DatasetGraph dsg, String backupfile) {
+        if ( transactional == null )
+            transactional = new TransactionalNull() ;
+        transactional.begin(ReadWrite.READ);
+        try {
+            Backup.backup(dsg, backupfile) ;
+        }
+        finally {
+            transactional.end() ;
+        }
+    }
+    
+    /** Perform a backup.
+     * 
+     * @see #backup(Transactional, DatasetGraph, String)
+     */
+    private static void backup(DatasetGraph dsg, String backupfile) {
         if ( !backupfile.endsWith(".nq") )
             backupfile = backupfile + ".nq" ;
 
@@ -84,7 +106,6 @@ public class Backup
                 out = new FileOutputStream(backupfile) ;
                 out = new BufferedOutputStream(out) ;
             }
-
             RDFDataMgr.write(out, dsg, Lang.NQUADS) ;
             out.close() ;
             out = null ;


[2/4] jena git commit: JENA-895 : Do backup inside a read transaction.

Posted by an...@apache.org.
JENA-895 : Do backup inside a read transaction.

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

Branch: refs/heads/master
Commit: a8caf7f8f514ed60f7a8f252acb13b05d164cfc1
Parents: 3ecbdba
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Mar 7 16:32:50 2015 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Mar 7 16:32:50 2015 +0000

----------------------------------------------------------------------
 .../org/apache/jena/fuseki/mgt/ActionBackup.java  | 18 ++++--------------
 1 file changed, 4 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/a8caf7f8/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/ActionBackup.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/ActionBackup.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/ActionBackup.java
index 34f134e..21679f0 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/ActionBackup.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/ActionBackup.java
@@ -28,8 +28,6 @@ import org.apache.jena.fuseki.servlets.ServletOps ;
 import org.slf4j.Logger ;
 import org.slf4j.LoggerFactory ;
 
-import com.hp.hpl.jena.sparql.core.DatasetGraph ;
-
 public class ActionBackup extends ActionAsyncTask
 {
     public ActionBackup() { super() ; }
@@ -48,24 +46,16 @@ public class ActionBackup extends ActionAsyncTask
             ServletOps.errorOccurred("Null for dataset name in item request");
             return null ;
         }
+        
         action.log.info(format("[%d] Backup dataset %s", action.id, name)) ;
         return new BackupTask(action) ;
     }
 
-    static class BackupTask implements Runnable {
+    static class BackupTask extends TaskBase {
         static private Logger log = LoggerFactory.getLogger("Backup") ;
         
-        private final long actionId ;
-        private final DatasetGraph dataset ;
-        private final String datasetName ;
-        
         public BackupTask(HttpAction action) {
-            this.actionId = action.id ;
-            action.getDataAccessPoint() ;
-            action.getDataAccessPoint().getDataService() ;
-            action.getDataAccessPoint().getDataService().getDataset() ;
-            this.dataset = action.getDataAccessPoint().getDataService().getDataset() ;
-            this.datasetName = action.getDatasetName() ;
+            super(action) ;
         }
 
         @Override
@@ -73,7 +63,7 @@ public class ActionBackup extends ActionAsyncTask
             try {
                 String backupFilename = Backup.chooseFileName(datasetName) ;
                 log.info(format("[%d] >>>> Start backup %s -> %s", actionId, datasetName, backupFilename)) ;
-                Backup.backup(dataset, backupFilename) ;
+                Backup.backup(transactional, dataset, backupFilename) ;
                 log.info(format("[%d] <<<< Finish backup %s -> %s", actionId, datasetName, backupFilename)) ;
             } catch (Exception ex) {
                 log.info(format("[%d] **** Exception in backup", actionId), ex) ;


[3/4] jena git commit: Abstract out a carrier of task information.

Posted by an...@apache.org.
Abstract out a carrier of task information.

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

Branch: refs/heads/master
Commit: 2fce000a96182d91c0345ab1cdca95a1706e799e
Parents: a8caf7f
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Mar 7 16:33:35 2015 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Mar 7 16:33:35 2015 +0000

----------------------------------------------------------------------
 .../org/apache/jena/fuseki/mgt/TaskBase.java    | 44 ++++++++++++++++++++
 1 file changed, 44 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/2fce000a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/TaskBase.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/TaskBase.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/TaskBase.java
new file mode 100644
index 0000000..1f3b3d3
--- /dev/null
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/mgt/TaskBase.java
@@ -0,0 +1,44 @@
+/**
+ * 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.fuseki.mgt;
+
+import org.apache.jena.fuseki.servlets.HttpAction ;
+
+import com.hp.hpl.jena.sparql.core.DatasetGraph ;
+import com.hp.hpl.jena.sparql.core.Transactional ;
+
+/** Base of async tasks - this caries some useful information aroud, leaving the
+ * implementation of Callable.run() to the specific task.
+ */
+abstract class TaskBase implements Runnable {
+    public final long actionId ;
+    public final DatasetGraph dataset ;
+    public final String datasetName ;
+    public final Transactional transactional ;
+    
+    protected TaskBase(HttpAction action) {
+        // The action is closed as part of action processing so is not
+        // available in the async task. Anything from it that is needed,
+        // taken out here.
+        this.actionId = action.id ;
+        this.dataset = action.getDataset() ;
+        this.transactional = action.getTransactional() ; 
+        this.datasetName = action.getDatasetName() ;
+    }
+}


[4/4] jena git commit: Initialization for admin tasks (specifically, backup).

Posted by an...@apache.org.
Initialization for admin tasks (specifically, backup).

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

Branch: refs/heads/master
Commit: 6d17e32dca1aa4c98b44c868b4d4efd2062fbb6b
Parents: 2fce000
Author: Andy Seaborne <an...@apache.org>
Authored: Sat Mar 7 16:34:06 2015 +0000
Committer: Andy Seaborne <an...@apache.org>
Committed: Sat Mar 7 16:34:06 2015 +0000

----------------------------------------------------------------------
 .../apache/jena/fuseki/servlets/HttpAction.java | 35 ++++++++++++++++++--
 1 file changed, 33 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/jena/blob/6d17e32d/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java
----------------------------------------------------------------------
diff --git a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java
index 245ed0f..7c69924 100644
--- a/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java
+++ b/jena-fuseki2/jena-fuseki-core/src/main/java/org/apache/jena/fuseki/servlets/HttpAction.java
@@ -138,8 +138,22 @@ public class HttpAction
         if ( dService == null || dService.getDataset() == null )
             // Null does not happens for service requests, (it does for admin requests - call setControlRequest) 
             throw new FusekiException("Null DataService in the request action") ;
-        
-        this.dsg = dService.getDataset() ;
+        setDataset(dService.getDataset()) ;
+    }
+    
+    /** Minimum initialization using just a dataset.
+     * <p>
+     * the HTTP Action will change its transactional state and
+     * {@link Transactional} instance according to its base dataset graph.
+     * </p>
+     * <p>There is no associated DataAccessPoint or DataService set by this operation.</p>
+     *  
+     * @param dsg DatasetGraph
+     */
+    private void setDataset(DatasetGraph dsg) {
+        this.dsg = dsg ;
+        if ( dsg == null )
+            return ;
         DatasetGraph basedsg = unwrap(dsg) ;
 
         if ( isTransactional(basedsg) && isTransactional(dsg) ) {
@@ -154,13 +168,30 @@ public class HttpAction
         }
     }
     
+    /** Return the dataset, if any (may be null) */
+    public DatasetGraph getDataset() {
+        return dsg ;
+    }
+
     public void setControlRequest(DataAccessPoint dataAccessPoint, String datasetUri) {
         this.dataAccessPoint = dataAccessPoint ;
         this.dataService = null ;
+        if ( dataAccessPoint != null )
+            this.dataService = dataAccessPoint.getDataService() ;
         this.datasetName = datasetUri ;
+        if ( dataService != null )
+            setDataset(dataAccessPoint.getDataService().getDataset()) ; 
     }
     
     /**
+     * Return the "Transactional" for this HttpAction.
+     */
+    
+    public Transactional getTransactional() {
+        return transactional ;
+    }
+
+    /**
      * Returns <code>true</code> iff the given {@link DatasetGraph} is an instance of {@link Transactional},
      * <code>false otherwise</code>.
      *