You are viewing a plain text version of this content. The canonical link for it is here.
Posted to derby-commits@db.apache.org by oy...@apache.org on 2007/09/05 15:38:09 UTC

svn commit: r572951 - in /db/derby/code/trunk/java: engine/org/apache/derby/ engine/org/apache/derby/database/ engine/org/apache/derby/iapi/services/replication/ engine/org/apache/derby/iapi/services/replication/master/ engine/org/apache/derby/iapi/sto...

Author: oysteing
Date: Wed Sep  5 06:38:05 2007
New Revision: 572951

URL: http://svn.apache.org/viewvc?rev=572951&view=rev
Log:
DERBY-2977: Add the basic code to Derby so that the master controller
            can be booted as a service when a "startmaster" command is
            issued to NetworkServerControl 
Contributed by Jørgen Løland

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/master/
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/master/MasterFactory.java   (with props)
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
    db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml
    db/derby/code/trunk/java/engine/org/apache/derby/modules.properties
    db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/database/Database.java Wed Sep  5 06:38:05 2007
@@ -116,6 +116,18 @@
 	public void disableLogArchiveMode(boolean deleteOnlineArchivedLogFiles) 
 		throws SQLException;
 
+    /**
+     * Start the replication master role for this database
+     * @param host The hostname for the slave
+     * @param port The port the slave is listening on
+     * @param replicationMode The type of replication contract.
+     * Currently only asynchronous replication is supported, but
+     * 1-safe/2-safe/very-safe modes may be added later.
+     * @exception SQLException Thrown on error
+     */
+    public void startReplicationMaster(String host, int port, 
+                                       String replicationMode)
+        throws SQLException;
 
 	/**
 	  * Freeze the database temporarily so a backup can be taken.

Added: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/master/MasterFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/master/MasterFactory.java?rev=572951&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/master/MasterFactory.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/master/MasterFactory.java Wed Sep  5 06:38:05 2007
@@ -0,0 +1,154 @@
+/*
+ 
+   Derby - Class
+   org.apache.derby.iapi.services.replication.master.MasterFactory
+ 
+   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.derby.iapi.services.replication.master;
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.reference.Property;
+
+import org.apache.derby.iapi.store.raw.RawStoreFactory;
+import org.apache.derby.iapi.store.raw.log.LogFactory;
+import org.apache.derby.iapi.store.raw.data.DataFactory;
+
+
+/**
+ * <p> 
+ * This is the interface for the replication master controller
+ * service. The master controller service is booted when this instance
+ * of Derby will have the replication master role for this database.
+ * </p> 
+ * <p> 
+ * The replication master service is responsible for managing all
+ * replication related functionality on the master side of replication.
+ * This includes connecting to the slave, setting up a log buffer to
+ * temporarily store log records from the LogFactory, and to ship
+ * these log records to the slave.
+ * </p> 
+ * <p> 
+ * The master controller currently only supports asynchronous
+ * replication. This means that there are no guarantees that
+ * transactions that have committed here (the master side) are also
+ * reflected on the slave side. However, the slave version of the
+ * database IS guaranteed to be transaction consistent. This implies
+ * that: <br>
+ *
+ * <ul>
+ *  <li>A transaction t that is committed on the master will either be
+ *  fully reflected or not be reflected at all on the slave when the
+ *  slave database is turned into a non-replicated database (that is,
+ *  at failover time)</li>
+ *
+ *  <li>Slave execution of operations is in the same serial order as
+ *  on the master because replication is based on redoing log records
+ *  to the slave. By definition, log records are in serial order. This
+ *  implies that if transaction t1 commits before t2 on the master,
+ *  and t2 has been committed on the slave, t1 is also guaranteed to
+ *  have committed on the slave.</li>
+ * </ul>
+ * </p>
+ */
+public interface MasterFactory {
+
+    /** The name of the Master Factory, used to boot the service.  */
+    public static final String MODULE =
+        "org.apache.derby.iapi.services.replication.master.MasterFactory";
+
+    /* Property names that are used as key values in the Properties objects*/
+
+    /** Property key used to specify which slave host to connect to */
+    public static final String SLAVE_HOST =
+        Property.PROPERTY_RUNTIME_PREFIX + "replication.master.slavehost";
+
+    /** Property key to specify which slave port to connect to */
+    public static final String SLAVE_PORT =
+        Property.PROPERTY_RUNTIME_PREFIX + "replication.master.slaveport";
+
+    /** Property key to specify replication mode */
+    public static final String REPLICATION_MODE =
+        Property.PROPERTY_RUNTIME_PREFIX + "replication.master.mode";
+
+    /* Property values */
+
+    /**
+     * Property value used to indicate that the service should be
+     * booted in asynchronous replication mode.
+     */
+    public static final String ASYNCHRONOUS_MODE =
+        Property.PROPERTY_RUNTIME_PREFIX + "asynch";
+
+
+    /* Methods */
+
+    /**
+     * Will perform all the work that is needed to set up replication
+     *
+     * @param rawStore The RawStoreFactory for the database
+     * @param dataFac The DataFactory for this database
+     * @param logFac The LogFactory ensuring recoverability for this database
+     */
+    public void startMaster(RawStoreFactory rawStore,
+                            DataFactory dataFac, LogFactory logFac);
+
+    /**
+     * Will perform all work that is needed to shut down replication
+     */
+    public void stopMaster();
+
+    /**
+     * Append a single log record to the replication log buffer.
+     *
+     * @param instant               the log address of this log record.
+     * @param dataLength            number of bytes in data[]
+     * @param dataOffset            offset in data[] to start copying from.
+     * @param optionalDataLength    number of bytes in optionalData[]
+     * @param optionalDataOffset    offset in optionalData[] to start copy from
+     * @param data                  "from" array to copy "data" portion of rec
+     * @param optionalData          "from" array to copy "optional data" from
+     **/
+    public void appendLogRecord(long instant, int dataLength,
+                                int dataOffset, int optionalDataLength,
+                                int optionalDataOffset,
+                                byte[] data, byte[] optionalData);
+
+    /**
+     * Used by the LogFactory to notify the replication master
+     * controller that the log records up to this instant have been
+     * flushed to disk. The master controller takes action according
+     * to the current replication strategy when this method is called.
+     *
+     * When the asynchronous replication strategy is used, the method
+     * does not force log shipping to the slave; the log records may
+     * be shipped now or later at the MasterFactory's discretion.
+     *
+     * However, if another strategy like 2-safe replication is
+     * implemented in the future, a call to this method may force log
+     * shipment before returning control to the caller.
+     *
+     * Currently, only asynchronous replication is supported.
+     *
+     * @param instant The highest log instant that has been flushed to
+     * disk
+     * @see LogFactory#flush
+     */
+    public void flushedTo(long instant);
+
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/master/MasterFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/access/AccessFactory.java Wed Sep  5 06:38:05 2007
@@ -214,6 +214,20 @@
      **************************************************************************
      */
 
+    /**
+     * Start the replication master role for this database
+     * @param host The hostname for the slave
+     * @param port The port the slave is listening on
+     * @param replicationMode The type of replication contract.
+     * Currently only asynchronous replication is supported, but
+     * 1-safe/2-safe/very-safe modes may be added later.
+     * @exception StandardException Standard Derby exception policy,
+     * thrown on error.
+     */
+    public void startReplicationMaster(String host, int port,
+                                       String replicationMode)
+        throws StandardException;
+
 	/**
 	  * Freeze the database temporarily so a backup can be taken.
 	  * <P>Please see Derby on line documentation on backup and restore.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/store/raw/RawStoreFactory.java Wed Sep  5 06:38:05 2007
@@ -681,6 +681,20 @@
 	 */
 	public TransactionInfo[] getTransactionInfo();
 
+    /**
+     * Start the replication master role for this database
+     * @param host The hostname for the slave
+     * @param port The port the slave is listening on
+     * @param replicationMode The type of replication contract.
+     * Currently only asynchronous replication is supported, but
+     * 1-safe/2-safe/very-safe modes may be added later.
+     * @exception StandardException Standard Derby exception policy,
+     * thrown on error.
+     */
+    public void startReplicationMaster(String host, int port,
+                                       String replicationMode)
+        throws StandardException;
+
 	/**
 	  * Freeze the database temporarily so a backup can be taken.
 	  * <P>Please see Derby on line documentation on backup and restore.

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/db/BasicDatabase.java Wed Sep  5 06:38:05 2007
@@ -349,6 +349,25 @@
 		return this.authenticationService;
 	}
 
+    /**
+     * Start the replication master role for this database
+     * @param host The hostname for the slave
+     * @param port The port the slave is listening on
+     * @param replicationMode The type of replication contract.
+     * Currently only asynchronous replication is supported, but
+     * 1-safe/2-safe/very-safe modes may be added later.
+     * @exception SQLException Thrown on error
+     */
+    public void startReplicationMaster(String host, int port,
+                                       String replicationMode)
+        throws SQLException {
+        try {
+            af.startReplicationMaster(host, port, replicationMode);
+        } catch (StandardException se) {
+            throw PublicAPI.wrapStandardException(se);
+        }
+    }
+
 	public void freeze() throws SQLException
 	{
 		try {

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java?rev=572951&view=auto
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java Wed Sep  5 06:38:05 2007
@@ -0,0 +1,270 @@
+/*
+ 
+   Derby - Class
+   org.apache.derby.impl.services.replication.master.MasterController
+ 
+   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.derby.impl.services.replication.master;
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.services.monitor.ModuleControl;
+import org.apache.derby.iapi.services.monitor.ModuleSupportable;
+
+import org.apache.derby.iapi.store.raw.RawStoreFactory;
+import org.apache.derby.iapi.store.raw.log.LogFactory;
+import org.apache.derby.iapi.store.raw.data.DataFactory;
+
+import org.apache.derby.iapi.services.replication.master.MasterFactory;
+import org.apache.derby.impl.services.replication.buffer.ReplicationLogBuffer;
+import org.apache.derby.impl.services.replication.buffer.LogBufferFullException;
+
+import java.util.Properties;
+
+/**
+ * <p> 
+ * This is an implementation of the replication master controller
+ * service. The service is booted when this instance of Derby will
+ * have the replication master role for this database.
+ * </p> 
+ * <p>
+ * Note: The current version of the class is far from complete. Code
+ * to control the replication master behavior will be added as more
+ * parts of the replication functionality is added to Derby. 
+ * </p>
+ *
+ * @see MasterFactory
+ */
+public class MasterController implements MasterFactory, ModuleControl,
+                                         ModuleSupportable {
+
+    private static final int DEFAULT_LOG_BUFFER_SIZE = 32768; //32K
+
+    private RawStoreFactory rawStoreFactory;
+    private DataFactory dataFactory;
+    private LogFactory logFactory;
+    private ReplicationLogBuffer logBuffer;
+    // waiting for code to go into trunk:
+    //    private LogShipper logShipper; 
+    //    private NetworkTransmit connection; 
+
+    private String replicationMode;
+    private String slavehost;
+    private int slaveport;
+
+
+
+    /**
+     * Empty constructor required by Monitor.bootServiceModule
+     */
+    public MasterController() { }
+
+    ////////////////////////////////////////////////////////////
+    // Implementation of methods from interface ModuleControl //
+    ////////////////////////////////////////////////////////////
+
+    /**
+     * Used by Monitor.bootServiceModule to start the service. Will:
+     *
+     * Set up basic variables
+     * Connect to the slave using the network service (DERBY-2921)
+     *
+     * Not implemented yet
+     *
+     * @param create Currently ignored
+     * @param properties Properties used to start the service in the
+     * correct mode
+     * @exception StandardException Standard Derby exception policy,
+     * thrown on error.
+     */
+    public void boot(boolean create, Properties properties)
+        throws StandardException {
+
+        replicationMode =
+            properties.getProperty(MasterFactory.REPLICATION_MODE);
+
+        slavehost = properties.getProperty(MasterFactory.SLAVE_HOST);
+
+        String port = properties.getProperty(MasterFactory.SLAVE_PORT);
+        if (port != null) {
+            slaveport = new Integer(port).intValue();
+        }
+
+        // Added when Network Service has been committed to trunk
+        // connection = new NetworkTransmit();
+
+        System.out.println("MasterController booted");
+    }
+
+    /**
+     * Will stop the replication master service
+     *
+     * Not implemented yet
+     */
+    public void stop() { }
+
+    ////////////////////////////////////////////////////////////////
+    // Implementation of methods from interface ModuleSupportable //
+    ////////////////////////////////////////////////////////////////
+
+    /**
+     * Used by Monitor.bootServiceModule to check if this class is
+     * usable for replication. To be usable, we require that
+     * asynchronous replication is specified in startParams by
+     * checking that a property with key
+     * MasterFactory.REPLICATION_MODE has the value
+     * MasterFactory.ASYNCHRONOUS_MODE. 
+     * @param startParams The properties used to boot replication
+     * @return true if asynchronous replication is requested, meaning
+     * that this MasterController is a suitable implementation for the
+     * MasterFactory service. False otherwise
+     * @see ModuleSupportable#canSupport 
+     */
+    public boolean canSupport(Properties startParams) {
+        String modeParam =
+            startParams.getProperty(MasterFactory.REPLICATION_MODE);
+
+        // currently only one attribute: asynchronous replication mode
+        if (modeParam != null && 
+            modeParam.equals(MasterFactory.ASYNCHRONOUS_MODE)) {
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    ////////////////////////////////////////////////////////////
+    // Implementation of methods from interface MasterFactory //
+    ////////////////////////////////////////////////////////////
+
+    /**
+     * Will perform all the work that is needed to set up replication
+     *
+     * Not implemented yet
+     *
+     * @param rawStore The RawStoreFactory for the database
+     * @param dataFac The DataFactory for this database
+     * @param logFac The LogFactory ensuring recoverability for this database
+     */
+    public void startMaster(RawStoreFactory rawStore,
+                            DataFactory dataFac, LogFactory logFac) {
+        // Added when Network Service has been committed to trunk:
+        // connection.connect(); // sets up a network connection to the slave
+
+        rawStoreFactory = rawStore;
+        dataFactory = dataFac;
+        logFactory = logFac;
+        logBuffer = new ReplicationLogBuffer(DEFAULT_LOG_BUFFER_SIZE);
+        //  logFactory.setReplicationMaster(this); //added later
+
+        if (replicationMode.equals(MasterFactory.ASYNCHRONOUS_MODE)) {
+            System.out.println("MasterController would now " +
+                               "start asynchronous log shipping");
+            // Added when Master Log Shipping code has been committed to trunk:
+            // logShipper = new AsynchronousLogShipper(connection);
+        }
+
+        // Add code that initializes replication by sending the
+        // database to the slave, making logFactory add logrecords to
+        // the buffer etc. Repliation should be up and running when
+        // this method returns.
+
+        System.out.println("MasterController started");
+    }
+
+    /**
+     * Will perform all work that is needed to shut down replication
+     *
+     * Not implemented yet
+     */
+    public void stopMaster() {
+        // logFactory.stopReplicationLogging(); // added later
+
+        // Added when Network Service has been committed to trunk:
+        // if (connection.isUp()) {
+        //     logShipper.flushAllLog();
+        // }
+
+        // logBuffer.stop();
+        System.out.println("MasterController stopped");
+    }
+
+    /**
+     * Append a single log record to the replication log buffer.
+     *
+     * @param instant               the log address of this log record.
+     * @param dataLength            number of bytes in data[]
+     * @param dataOffset            offset in data[] to start copying from.
+     * @param optionalDataLength    number of bytes in optionalData[]
+     * @param optionalDataOffset    offset in optionalData[] to start copy from
+     * @param data                  "from" array to copy "data" portion of rec
+     * @param optionalData          "from" array to copy "optional data" from
+     *
+     **/
+    public void appendLogRecord(long instant,
+                                int dataLength,
+                                int dataOffset,
+                                int optionalDataLength,
+                                int optionalDataOffset,
+                                byte[] data,
+                                byte[] optionalData) {
+        try {
+            logBuffer.appendLogRecord(instant, dataLength, dataOffset,
+                                      optionalDataLength, optionalDataOffset,
+                                      data, optionalData);
+        } catch (LogBufferFullException lbfe) {
+            // Waiting for log shipper to implement this
+            // We have multiple alternatives: 
+            //  1) Try to force-send some log to the slave:
+            //     logShipper.forceFlush()
+            //  2) Increase the size of the buffer
+            // Stop replication if both these are unsuccessful or not
+            // an alternative. 
+        }
+    }
+
+    /**
+     * Used by the LogFactory to notify the replication master
+     * controller that the log records up to this instant have been
+     * flushed to disk. The master controller takes action according
+     * to the current replication strategy when this method is called.
+     *
+     * When the asynchronous replication strategy is used, the method
+     * does not force log shipping to the slave; the log records may
+     * be shipped now or later at the MasterController's discretion.
+     *
+     * However, if another strategy like 2-safe replication is
+     * implemented in the future, a call to this method may force log
+     * shipment before returning control to the caller.
+     *
+     * Currently, only asynchronous replication is supported.
+     *
+     * Not implemented yet
+     *
+     * @param instant The highest log instant that has been flushed to
+     * disk
+     *
+     * @see MasterFactory#flushedTo
+     * @see LogFactory#flush
+     */
+    public void flushedTo(long instant) {
+        // logShipper.flushedTo(instant); 
+    }
+
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/access/RAMAccessManager.java Wed Sep  5 06:38:05 2007
@@ -912,6 +912,22 @@
 		return rawstore.getTransactionInfo();
 	}
 
+    /**
+     * Start the replication master role for this database. 
+     * @param host The hostname for the slave
+     * @param port The port the slave is listening on
+     * @param replicationMode The type of replication contract.
+     * Currently only asynchronous replication is supported, but
+     * 1-safe/2-safe/very-safe modes may be added later.
+     * @exception StandardException Standard Derby exception policy,
+     * thrown on error.
+     */
+    public void startReplicationMaster(String host, int port,
+                                       String replicationMode)
+        throws StandardException {
+        rawstore.startReplicationMaster(host, port, replicationMode);
+    }
+
 	public void freeze() throws StandardException
 	{
 		rawstore.freeze();

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/RawStore.java Wed Sep  5 06:38:05 2007
@@ -88,6 +88,7 @@
 import java.security.PrivilegedExceptionAction;
 import java.lang.SecurityException;
 
+import org.apache.derby.iapi.services.replication.master.MasterFactory;
 
 /**
 	A Raw store that implements the RawStoreFactory module by delegating all the
@@ -440,6 +441,40 @@
 		logFactory.checkpoint(this, dataFactory, xactFactory, false);
 	}
 
+    /**
+     * Start the replication master role for this database
+     * @param host The hostname for the slave
+     * @param port The port the slave is listening on
+     * @param replicationMode The type of replication contract.
+     * Currently only asynchronous replication is supported, but
+     * 1-safe/2-safe/very-safe modes may be added later.
+     * @exception StandardException Standard Derby exception policy,
+     * thrown on error.
+     */
+    public void startReplicationMaster(String host, int port,
+                                       String replicationMode)
+        throws StandardException {
+
+        if (isReadOnly()) {
+            throw StandardException.newException(
+                      SQLState.CANNOT_REPLICATE_READONLY_DATABASE);
+        }
+
+        Properties replicationProps = new Properties();
+        replicationProps.setProperty(MasterFactory.SLAVE_HOST, host);
+        replicationProps.setProperty(MasterFactory.SLAVE_PORT,
+                                     new Integer(port).toString());
+
+        replicationProps.setProperty(MasterFactory.REPLICATION_MODE,
+                                     replicationMode);
+
+        MasterFactory masterFactory = (MasterFactory)
+            Monitor.bootServiceModule(true, this, getMasterFactoryModule(),
+                                      replicationProps);
+        masterFactory.startMaster(this, dataFactory, logFactory);
+
+    }
+
 	public void freeze() throws StandardException
 	{
 		logFactory.checkpoint(this, dataFactory, xactFactory, true);
@@ -1988,6 +2023,11 @@
 	{
 		return TransactionFactory.MODULE;
 	}
+
+    public String getMasterFactoryModule()
+    {
+        return MasterFactory.MODULE;
+    }
 
 	public String getDataFactoryModule()
 	{

Modified: db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml Wed Sep  5 06:38:05 2007
@@ -4670,6 +4670,10 @@
             <title>Class XRE: Replication Exceptions</title>
 
             <msg>
+                <name>XRE00</name>
+                <text>Cannot start replication for a read-only database.</text>
+            </msg>
+            <msg>
                 <name>XRE01</name>
                 <text>The log received from the master is corrupted.</text>
             </msg>

Modified: db/derby/code/trunk/java/engine/org/apache/derby/modules.properties
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/modules.properties?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/modules.properties (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/modules.properties Wed Sep  5 06:38:05 2007
@@ -306,6 +306,9 @@
 derby.module.rawStore=org.apache.derby.impl.store.raw.RawStore
 cloudscape.config.rawStore=derby
 
+# Replication
+derby.module.replication.master=org.apache.derby.impl.services.replication.master.MasterController
+cloudscape.config.replication.master=derby
 
 #
 # Support for read-only databases

Modified: db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java?rev=572951&r1=572950&r2=572951&view=diff
==============================================================================
--- db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java (original)
+++ db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/SQLState.java Wed Sep  5 06:38:05 2007
@@ -1750,6 +1750,7 @@
     /*
     ** Replication XRExx
     */
+    String CANNOT_REPLICATE_READONLY_DATABASE                      = "XRE00";
     String REPLICATION_LOG_CORRUPTED                               = "XRE01";
 
 }