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 2008/02/06 10:56:03 UTC

svn commit: r618944 - in /db/derby/code/trunk/java: engine/org/apache/derby/iapi/services/replication/slave/ engine/org/apache/derby/impl/db/ engine/org/apache/derby/impl/services/replication/master/ engine/org/apache/derby/impl/services/replication/ne...

Author: oysteing
Date: Wed Feb  6 01:55:58 2008
New Revision: 618944

URL: http://svn.apache.org/viewvc?rev=618944&view=rev
Log:
DERBY-3361: Make the startSlave connection wait until slave replication mode is confirmed started or until an error occurs.
In addition, this patch adds host and port information to a derby.log message and a few exceptions for easier debugging. DERBY-3356 is also fixed by this patch.
Contributed by Jorgen Loland


Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/slave/SlaveFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/db/SlaveDatabase.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/ReplicationMessageReceive.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/SlaveAddress.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/slave/SlaveController.java
    db/derby/code/trunk/java/engine/org/apache/derby/loc/messages.xml
    db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/MessageId.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/slave/SlaveFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/slave/SlaveFactory.java?rev=618944&r1=618943&r2=618944&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/slave/SlaveFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/iapi/services/replication/slave/SlaveFactory.java Wed Feb  6 01:55:58 2008
@@ -139,5 +139,14 @@
      * @see org.apache.derby.impl.services.replication.master.MasterController#flushedTo
      */
     public void failover();
+    
+    /**
+     * Check whether or not slave replication mode has been
+     * successfully started.
+     *
+     * @return true if slave replication mode has started
+     * successfully, false if slave mode startup is not yet confirmed
+     */
+    public boolean isStarted();
 
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/db/SlaveDatabase.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/db/SlaveDatabase.java?rev=618944&r1=618943&r2=618944&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/db/SlaveDatabase.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/db/SlaveDatabase.java Wed Feb  6 01:55:58 2008
@@ -31,7 +31,6 @@
 import org.apache.derby.iapi.services.monitor.Monitor;
 import org.apache.derby.iapi.services.replication.slave.SlaveFactory;
 import org.apache.derby.iapi.sql.conn.LanguageConnectionContext;
-import org.apache.derby.impl.services.monitor.UpdateServiceProperties;
 
 import java.sql.Driver;
 import java.sql.DriverManager;
@@ -74,6 +73,16 @@
      * executed for this database */
     private volatile boolean inReplicationSlaveMode;
     private volatile boolean shutdownInitiated;
+
+    /** True until this database has been successfully booted. Any
+     * exception that occurs while inBoot is true will be handed to
+     * the client thread booting this database. */
+    private volatile boolean inBoot;
+
+    /** Set by the database boot thread if it fails before slave mode
+     * has been started properly (i.e., if inBoot is true). This
+     * exception will then be reported to the client connection. */
+    private volatile StandardException bootException;
     private String dbname; // The name of the replicated database
     private volatile SlaveFactory slaveFac;
 
@@ -109,6 +118,7 @@
         throws StandardException {
 
         inReplicationSlaveMode = true;
+        inBoot = true;
         shutdownInitiated = false;
 
         dbname = startParams.getProperty(SlaveFactory.SLAVE_DB);
@@ -118,21 +128,10 @@
             new SlaveDatabaseBootThread(create, startParams);
         new Thread(dbBootThread).start();
 
-        try {
-            // We cannot claim to be booted until the storage factory
-            // has been set in the startParams because
-            // TopService.bootModule (the caller of this method) uses
-            // the storage factory object. The storage factory is set
-            // in RawStore.boot, and we have to wait for this to
-            // happen.
-            UpdateServiceProperties usp =
-                (UpdateServiceProperties) startParams;
-            while (usp.getStorageFactory() == null){
-                Thread.sleep(500);
-            }
-        } catch (Exception e) {
-            //Todo: report exception to derby.log
-        }
+        // Check that the database was booted successfully, or throw
+        // the exception that caused the boot to fail.
+        verifySuccessfulBoot();
+        inBoot = false;
 
         // This module has now been booted (hence active=true) even
         // though submodules like store and authentication may not
@@ -168,10 +167,6 @@
     public SlaveDatabase() {
     }
 
-    public void setSlaveFactory(SlaveFactory f) {
-        slaveFac = f;
-    }
-
     ////////////////////////
     // Database interface //
     ////////////////////////
@@ -297,16 +292,80 @@
             } catch (StandardException se) {
                 // We get here when SlaveController#stopSlave has been
                 // called, or if a fatal exception has been thrown.
-                handleShutdown ();
+                handleShutdown(se);
             }
         }
     }
 
+    ////////////////////
+    // Private Methods//
+    ////////////////////
+
     /**
-     * Shutdown this database
+     * Verify that the slave functionality has been properly started.
+     * This method will block until a successful slave startup has
+     * been confirmed, or it will throw the exception that caused it
+     * to fail.
+     */
+    private void verifySuccessfulBoot() throws StandardException {
+        while (!(isSlaveFactorySet() && slaveFac.isStarted())) {
+            if (bootException != null) {
+                throw bootException;
+            } else {
+                try {
+                    Thread.sleep(500);
+                } catch (InterruptedException ie) {
+                    // do nothing
+                }
+            }
+        }
+    }
+
+    /** 
+     * If slaveFac (the reference to the SlaveFactory) has not already
+     * been set, this method will try to set it by calling
+     * Monitor.findServiceModule. If slavFac was already set, the
+     * method does not do anything.
+     *
+     * @return true if slaveFac is set after calling this method,
+     * false otherwise
      */
-    private void handleShutdown() {
+    private boolean isSlaveFactorySet() {
+        if (slaveFac != null) {
+            return true;
+        }
 
+        try {
+            slaveFac = (SlaveFactory)Monitor.
+                findServiceModule(this, SlaveFactory.MODULE);
+            return true;
+        } catch (StandardException se) {
+            // We get a StandardException if SlaveFactory has not been 
+            // booted yet. Safe to retry later.
+            return false;
+        }
+    }
+
+    /**
+     * Used to shutdown this database. 
+     *
+     * If an error occurs as part of the database boot process, we
+     * hand the exception that caused boot to fail to the client
+     * thread. The client thread will in turn shut down this database.
+     *
+     * If an error occurs at a later stage than during boot, we shut
+     * down the database by setting up a connection with the shutdown
+     * attribute. The internal connection is required because database
+     * shutdown requires EmbedConnection to do cleanup.
+     *
+     * @param shutdownCause the reason why the database needs to be
+     * shutdown
+     */
+    private void handleShutdown(StandardException shutdownCause) {
+        if (inBoot) {
+            bootException = shutdownCause;
+            return;
+        } 
         try {
             shutdownInitiated = true;
             String driverName = 

Modified: 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=618944&r1=618943&r2=618944&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/master/MasterController.java Wed Feb  6 01:55:58 2008
@@ -373,10 +373,12 @@
                     (SQLState.REPLICATION_MASTER_TIMED_OUT, dbname);
         } catch (IOException ioe) {
             throw StandardException.newException
-                    (SQLState.REPLICATION_CONNECTION_EXCEPTION, ioe, dbname);
+                    (SQLState.REPLICATION_CONNECTION_EXCEPTION, ioe, 
+                     dbname, slavehost, String.valueOf(slaveport));
         } catch (Exception e) {
             throw StandardException.newException
-                    (SQLState.REPLICATION_CONNECTION_EXCEPTION, e, dbname);
+                    (SQLState.REPLICATION_CONNECTION_EXCEPTION, e,
+                     dbname, slavehost, String.valueOf(slaveport));
         }
     }
     

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/ReplicationMessageReceive.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/ReplicationMessageReceive.java?rev=618944&r1=618943&r2=618944&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/ReplicationMessageReceive.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/ReplicationMessageReceive.java Wed Feb  6 01:55:58 2008
@@ -1,3 +1,4 @@
+
 /*
  
    Derby - Class org.apache.derby.impl.services.replication.net.ReplicationMessageReceive
@@ -30,7 +31,9 @@
 import java.security.PrivilegedExceptionAction;
 import javax.net.ServerSocketFactory;
 import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.reference.MessageId;
 import org.apache.derby.iapi.reference.SQLState;
+import org.apache.derby.iapi.services.monitor.Monitor;
 
 /**
  * This class is the Receiver (viz. Socket server or listener) part of the
@@ -64,17 +67,31 @@
      *                 the slave to replicate to.
      * @param portNumber an integer that contains the port number of the
      *                   slave to replicate to.
+     * @param dbname the name of the database
      *
      * @throws StandardException If an exception occurs while trying to
      *                           resolve the host name.
      */
-    public ReplicationMessageReceive(String hostName, int portNumber)
+    public ReplicationMessageReceive(String hostName, int portNumber, 
+                                     String dbname)
         throws StandardException {
         try {
             slaveAddress = new SlaveAddress(hostName, portNumber);
+            Monitor.logTextMessage(MessageId.REPLICATION_SLAVE_NETWORK_LISTEN, 
+                                   dbname, getHostName(), 
+                                   String.valueOf(getPort()));
         } catch (UnknownHostException uhe) {
+            // cannot use getPort because SlaveAddress creator threw
+            // exception and has therefore not been initialized
+            String port;
+            if (portNumber > 0) {
+                port = String.valueOf(portNumber);
+            } else {
+                port = String.valueOf(SlaveAddress.DEFAULT_PORT_NO);
+            }
             throw StandardException.newException
-                (SQLState.REPLICATION_CONNECTION_EXCEPTION, uhe);
+                (SQLState.REPLICATION_CONNECTION_EXCEPTION, uhe, 
+                 dbname, hostName, port);
         }
     }
     
@@ -248,4 +265,24 @@
         ClassNotFoundException, IOException {
         return (ReplicationMessage)socketConn.readMessage();
     }
+
+    /**
+     * Used to get the host name the slave listens for master
+     * connections on
+     *
+     * @return the host name 
+     */
+    public String getHostName() {
+        return slaveAddress.getHostAddress().getHostName();
+     }
+
+    /**
+     * Used to get the port number the slave listens for master
+     * connections on
+     *
+     * @return the port number
+     */
+    public int getPort() {
+        return slaveAddress.getPortNumber();
+     }
 }

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/SlaveAddress.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/SlaveAddress.java?rev=618944&r1=618943&r2=618944&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/SlaveAddress.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/net/SlaveAddress.java Wed Feb  6 01:55:58 2008
@@ -43,7 +43,7 @@
      * Used as the default port number if the port number
      * is not mentioned.
      */
-    private final int DEFAULT_PORT_NO = 8001;
+    public static final int DEFAULT_PORT_NO = 8001;
     
     /**
      *

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/slave/SlaveController.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/slave/SlaveController.java?rev=618944&r1=618943&r2=618944&view=diff
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/slave/SlaveController.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/services/replication/slave/SlaveController.java Wed Feb  6 01:55:58 2008
@@ -25,7 +25,6 @@
 import org.apache.derby.iapi.error.StandardException;
 import org.apache.derby.iapi.reference.Attribute;
 import org.apache.derby.iapi.reference.MessageId;
-import org.apache.derby.iapi.reference.Property;
 import org.apache.derby.iapi.reference.SQLState;
 import org.apache.derby.iapi.services.monitor.ModuleControl;
 import org.apache.derby.iapi.services.monitor.ModuleSupportable;
@@ -89,6 +88,14 @@
     // chunks received from the master.
     private volatile boolean inReplicationSlaveMode = true;
 
+    /** Whether or not this SlaveController has been successfully
+     * started, including setting up a connection with the master and
+     * starting the log receiver thread. The client connection that
+     * initiated slave replication mode on this database will not
+     * report that slave mode was successfully started (i.e., it will
+     * hang) until startupSuccessful has been set to true */
+    private volatile boolean startupSuccessful = false;
+
     // Used to parse chunks of log records received from the master.
     private ReplicationLogScan logScan;
 
@@ -192,10 +199,6 @@
     public void startSlave(RawStoreFactory rawStore, LogFactory logFac)
         throws StandardException {
 
-        slaveDb = (SlaveDatabase)
-                Monitor.findService(Property.DATABASE_MODULE, dbname);
-        slaveDb.setSlaveFactory(this);
-
         rawStoreFactory = rawStore;
 
         try {
@@ -213,7 +216,7 @@
         // Retry to setup a connection with the master until a
         // connection has been established or until we are no longer
         // in replication slave mode
-        receiver = new ReplicationMessageReceive(slavehost, slaveport);
+        receiver = new ReplicationMessageReceive(slavehost, slaveport, dbname);
         while (!setupConnection()) {
             if (!inReplicationSlaveMode) {
                 // If we get here, another thread has called
@@ -230,6 +233,7 @@
         logScan = new ReplicationLogScan();
 
         startLogReceiverThread();
+        startupSuccessful = true;
 
         Monitor.logTextMessage(MessageId.REPLICATION_SLAVE_STARTED, dbname);
     }
@@ -251,7 +255,9 @@
 
         try {
             // Unplug the replication network connection layer
-            receiver.tearDown(); 
+            if (receiver != null) {
+                receiver.tearDown(); 
+            }
         } catch (IOException ioe) {
             ReplicationLogger.logError(null, ioe, dbname);
         }
@@ -306,6 +312,13 @@
                 (MessageId.REPLICATION_FAILOVER_SUCCESSFUL, dbname);
     }
 
+    /**
+     * @see SlaveFactory#isStarted
+     */
+    public boolean isStarted() {
+        return startupSuccessful;
+    }
+
     ////////////////////////////////////////////////////////////
     // Private Methods                                        //
     ////////////////////////////////////////////////////////////
@@ -339,7 +352,8 @@
                 return false;
             } else {
                 throw StandardException.newException
-                    (SQLState.REPLICATION_CONNECTION_EXCEPTION, e, dbname);
+                    (SQLState.REPLICATION_CONNECTION_EXCEPTION, e,
+                    dbname, slavehost, String.valueOf(receiver.getPort()));
             }
         }
     }
@@ -424,7 +438,12 @@
             logError(MessageId.REPLICATION_FATAL_ERROR, e, dbname);
 
         // todo: notify master of the problem
-        // todo: rawStoreFactory.stopReplicationSlave();
+        try {
+            stopSlave();
+        } catch (StandardException se) {
+            ReplicationLogger.
+                logError(MessageId.REPLICATION_FATAL_ERROR, se, dbname);
+        }
     }
 
 
@@ -438,8 +457,6 @@
      */
     private class SlaveLogReceiverThread extends Thread {
         public void run() {
-            // Debug only - println will be removed
-            System.out.println("Started log receiver thread");
             try {
                 ReplicationMessage message;
                 while (inReplicationSlaveMode) {

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=618944&r1=618943&r2=618944&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 Feb  6 01:55:58 2008
@@ -4740,8 +4740,10 @@
 
             <msg>
                 <name>XRE04</name>
-                <text>Could not establish a connection to the peer of the replicated database '{0}'.</text>
+                <text>Could not establish a connection to the peer of the replicated database '{0}' on address '{1}:{2}'.</text>
                 <arg>dbname</arg>
+                <arg>hostname</arg>
+                <arg>portname</arg>
             </msg>
 
             <msg>
@@ -7653,6 +7655,14 @@
                 <name>R010</name>
                 <text>Replication master trying to reconnect to slave for database '{0}'.</text>
                 <arg>dbname</arg>
+            </msg>
+            
+            <msg>
+                <name>R011</name>
+                <text>Replication slave database '{0}' listens for connections from master on '{1}:{2}'.</text>
+                <arg>dbname</arg>
+                <arg>hostname</arg>
+                <arg>port</arg>
             </msg>
             
             <msg>

Modified: db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/MessageId.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/MessageId.java?rev=618944&r1=618943&r2=618944&view=diff
==============================================================================
--- db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/MessageId.java (original)
+++ db/derby/code/trunk/java/shared/org/apache/derby/shared/common/reference/MessageId.java Wed Feb  6 01:55:58 2008
@@ -183,6 +183,7 @@
     String REPLICATION_MASTER_STOPPED                    = "R008";
     String REPLICATION_LOGSHIPPER_EXCEPTION              = "R009";
     String REPLICATION_MASTER_RECONN                     = "R010";
+    String REPLICATION_SLAVE_NETWORK_LISTEN              = "R011";    
     String REPLICATION_FAILOVER_SUCCESSFUL               = "R020";
 
 }