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 ka...@apache.org on 2006/09/19 12:58:04 UTC

svn commit: r447815 - in /db/derby/code/trunk/java/engine/org/apache/derby/impl/store: build.xml raw/data/BaseDataFileFactory.java raw/data/RAFContainer.java raw/data/RAFContainer4.java raw/data/RAFContainerFactory.java

Author: kahatlen
Date: Tue Sep 19 03:58:03 2006
New Revision: 447815

URL: http://svn.apache.org/viewvc?view=rev&rev=447815
Log:
DERBY-801 (partial) Allow parallel access to data files

Adding RAFContainer4 which is used instead of RAFContainer on jdk1.4
and higher. RAFContainer4 uses FileChannels to access the data files,
and therefore multiple threads may access them in parallel.

Patch contributed by Anders Morken.

Added:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer4.java   (with props)
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainerFactory.java   (with props)
Modified:
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/build.xml
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
    db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/build.xml
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/build.xml?view=diff&rev=447815&r1=447814&r2=447815
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/build.xml (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/build.xml Tue Sep 19 03:58:03 2006
@@ -16,7 +16,7 @@
   limitations under the License.
 -->
 
-<project default="compile_impl_store" basedir="../../../../../../..">
+<project default="compile_impl_store_14" basedir="../../../../../../..">
 
 <!-- Set Properties -->
   <!-- User settings -->
@@ -53,6 +53,27 @@
         <pathelement path="${java13compile.classpath}"/>
       </classpath>
       <include name="${derby.dir}/impl/store/**"/>
+      <exclude name="${derby.dir}/impl/store/raw/data/RAFContainer4.java"/>
+    </javac>
+  </target>
+  <target name="compile_impl_store_14" depends="compile_impl_store">
+      <javac
+        source="1.4"
+        target="1.4"
+      bootclasspath="${empty}"
+      nowarn="on"
+      debug="${debug}"
+      depend="${depend}"
+      deprecation="${deprecation}"
+      optimize="${optimize}"
+      proceed="${proceed}"
+      verbose="${verbose}"
+      srcdir="${derby.engine.src.dir}"
+      destdir="${out.dir}">
+      <classpath>
+        <pathelement path="${java14compile.classpath}"/>
+      </classpath>
+      <include name="${derby.dir}/impl/store/raw/data/RAFContainer4.java"/>
     </javac>
   </target>
 </project>

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java?view=diff&rev=447815&r1=447814&r2=447815
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/BaseDataFileFactory.java Tue Sep 19 03:58:03 2006
@@ -235,6 +235,9 @@
     private File            backupRoot;
     private String[]        bfilelist;
 
+	// Creates RAFContainer instances tailored to the running VM
+	private RAFContainerFactory rafContainerFactory = new RAFContainerFactory();
+
 	/*
 	** Constructor
 	*/
@@ -1455,7 +1458,7 @@
     Cacheable newContainerObject()
     {
         if( supportsRandomAccess)
-            return new RAFContainer(this);
+            return rafContainerFactory.newRAFContainer(this);
         else
             return new InputStreamContainer( this);
     } 

Modified: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java?view=diff&rev=447815&r1=447814&r2=447815
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java (original)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer.java Tue Sep 19 03:58:03 2006
@@ -252,7 +252,7 @@
 		// RESOLVE: leaveStub false
 	}
 
-	final void closeContainer() {
+	void closeContainer() {
 
 		if (fileData != null) {
 			try {
@@ -472,7 +472,7 @@
      * page.
      * @return byte array of the the page data as it should be on the disk.
      */
-    private byte[] updatePageArray(long pageNumber, 
+    protected byte[] updatePageArray(long pageNumber, 
                                    byte[] pageData, 
                                    byte[] encryptionBuf, 
                                    boolean encryptWithNewEngine) 

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer4.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer4.java?view=auto&rev=447815
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer4.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer4.java Tue Sep 19 03:58:03 2006
@@ -0,0 +1,387 @@
+/*
+
+   Derby - Class org.apache.derby.impl.store.raw.data.RAFContainer4
+
+   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.store.raw.data;
+
+
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.derby.iapi.services.sanity.SanityManager;
+import org.apache.derby.iapi.services.io.FormatIdUtil;
+
+import org.apache.derby.impl.store.raw.data.BaseDataFileFactory;
+import org.apache.derby.iapi.store.raw.ContainerKey;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.channels.ClosedChannelException;
+
+/**
+ * RAFContainer4 overrides a few methods in RAFContainer in an attempt to use
+ * FileChannel from Java 1.4's New IO framework to issue multiple IO operations
+ * to the same file concurrently instead of strictly serializing IO operations
+ * using a mutex on the container object.
+ * <p>
+ * Note that our requests for multiple concurrent IOs may be serialized further
+ * down in the IO stack - this is entirely up to the JVM and OS. However, at
+ * least in Linux on Sun's 1.4.2_09 JVM we see the desired behavior:
+ * The FileChannel.read/write(ByteBuffer buf, long position) calls map to
+ * pread/pwrite system calls, which enable efficient IO to the same file
+ * descriptor by multiple threads.
+ * <p>
+ * This whole class should be merged back into RAFContainer when Derby
+ * officially stops supporting Java 1.3.
+ * <p>
+ * Significant behavior changes from RAFContainer:
+ * <ol>
+ * <li> Multiple concurrent IOs permitted.
+ * <li> State changes to the container (create, open, close) can now happen while
+ *      IO is in progress due to the lack of locking. Closing a container while
+ *      IO is in progress will cause IOExceptions in the thread calling readPage
+ *      or writePage. If this happens something is probably amiss anyway.
+ *      The iosInProgress variable is used in an attempt to detect this should it
+ *      happen while running a debug build.
+ * </ol>
+ *
+ * @see java.nio.channels.FileChannel
+ */
+class RAFContainer4 extends RAFContainer {
+
+    /**
+     * This channel will be retrieved from RAFContainer's fileData
+     * member when fileData is set. We wrap a couple of RAFContainer's methods
+     * to accomplish this.
+     */
+    private FileChannel ourChannel = null;
+
+    /**
+     * For debugging - will be incremented when an IO is started, decremented
+     * when it is done. Should be == 0 when container state is changed.
+     */
+    private int iosInProgress = 0;
+
+    public RAFContainer4(BaseDataFileFactory factory) {
+        super(factory);
+    }
+
+    /*
+     * Wrapping methods that retrieve the FileChannel from RAFContainer's
+     * fileData after calling the real methods in RAFContainer.
+     */
+    synchronized boolean openContainer(ContainerKey newIdentity)
+        throws StandardException
+    {
+        if (SanityManager.DEBUG) {
+            SanityManager.ASSERT(iosInProgress == 0,
+                    "Container opened while IO operations are in progress. "
+                    + "This should not happen.");
+        }
+
+        boolean result = super.openContainer(newIdentity);
+        if (result == true && super.fileData != null &&
+                super.fileData instanceof java.io.RandomAccessFile) {
+            /** XXX - this cast isn't testing friendly.
+             * A testing class that implements StorageRandomAccessFile but isn't
+             * a RandomAccessFile will be "worked around" by this class. An
+             * example of such a class is
+             * functionTests/util/corruptio/CorruptRandomAccessFile.java.
+             * An interface rework may be necessary.
+             */
+            ourChannel = ((RandomAccessFile)super.fileData).getChannel();
+        }
+        return result;
+    }
+
+    synchronized void createContainer(ContainerKey newIdentity)
+        throws StandardException
+    {
+        if (SanityManager.DEBUG) {
+            SanityManager.ASSERT(iosInProgress == 0,
+                    "Container created while IO operations are in progress. "
+                    + "This should not happen.");
+        }
+        super.createContainer(newIdentity);
+
+        if (super.fileData != null &&
+                super.fileData instanceof java.io.RandomAccessFile) {
+            // XXX - see "XXX" comment above.
+            ourChannel = ((RandomAccessFile) super.fileData).getChannel();
+        }
+    }
+
+
+    synchronized void closeContainer() {
+        if (SanityManager.DEBUG) {
+            SanityManager.ASSERT(iosInProgress == 0,
+                    "Container closed while IO operations are in progress. "
+                    + " This should not happen.");
+        }
+        if(ourChannel != null) {
+            try {
+                ourChannel.close();
+            } catch (IOException e) {
+                // nevermind.
+            } finally {
+                ourChannel=null;
+            }
+        }
+        super.closeContainer();
+    }
+
+    /**
+     * These are the methods that were rewritten to use FileChannel.
+     **/
+
+    /**
+     *  Read a page into the supplied array.
+     *
+     *  <BR> MT - thread safe
+     *  @exception IOException exception reading page
+     *  @exception StandardException Standard Cloudscape error policy
+     */
+    protected void readPage(long pageNumber, byte[] pageData)
+         throws IOException, StandardException
+    {
+        FileChannel ioChannel;
+        synchronized (this) {
+            ioChannel = ourChannel;
+            if (SanityManager.DEBUG) {
+                SanityManager.ASSERT(!getCommittedDropState());
+            }
+        }
+
+        if(ioChannel != null) {
+
+            long pageOffset = pageNumber * pageSize;
+
+            ByteBuffer pageBuf = ByteBuffer.wrap(pageData);
+
+            // I hope the try/finally is optimized away by the
+            // compiler/jvm when SanityManager.DEBUG == false?
+            try {
+                if (SanityManager.DEBUG) {
+                    synchronized(this) {
+                        iosInProgress++;
+                    }
+                }
+
+                readFull(pageBuf, ioChannel, pageOffset);
+            }
+            finally {
+                if (SanityManager.DEBUG) {
+                    synchronized(this) {
+                        iosInProgress--;
+                    }
+                }
+
+            }
+
+            if (dataFactory.databaseEncrypted() &&
+                pageNumber != FIRST_ALLOC_PAGE_NUMBER)
+            {
+                decryptPage(pageData, pageSize);
+            }
+        }
+        else
+        { // iochannel was not initialized, fall back to original method.
+            if(SanityManager.DEBUG) {
+                SanityManager.DEBUG_PRINT("RAFContainer4",
+                        "New style readPage attempted with uninitialized ioChannel");
+            }
+            super.readPage(pageNumber, pageData);
+        }
+    }
+
+
+    /**
+     *  Write a page from the supplied array.
+     *
+     *  <BR> MT - thread safe
+     *
+     *  @exception StandardException Standard Cloudscape error policy
+     *  @exception IOException IO error accessing page
+     */
+    protected void writePage(long pageNumber, byte[] pageData, boolean syncPage)
+         throws IOException, StandardException
+    {
+        FileChannel ioChannel;
+        synchronized(this) {
+            // committed and dropped, do nothing.
+            // This file container may only be a stub
+            if (getCommittedDropState())
+                return;
+            ioChannel = ourChannel;
+        }
+        if(ioChannel != null) {
+            ///////////////////////////////////////////////////
+            //
+            // RESOLVE: right now, no logical -> physical mapping.
+            // We can calculate the offset.  In the future, we may need to
+            // look at the allocation page or the in memory translation table
+            // to figure out where the page should go
+            //
+            /////////////////////////////////////////////////
+
+            long pageOffset = pageNumber * pageSize;
+
+            byte[] encryptionBuf = null;
+            // We only need to allocate the encryptionBuf if updatePageArray is
+            // actually going to use it.
+            if (dataFactory.databaseEncrypted()) {
+                encryptionBuf = new byte[pageSize];
+            }
+
+            byte[] dataToWrite = updatePageArray(pageNumber,
+                                                 pageData,
+                                                 encryptionBuf,
+                                                 false);
+
+            if (SanityManager.DEBUG) {
+                SanityManager.ASSERT(dataToWrite != null,
+                        "RAFContainer4: dataToWrite is null after updatePageArray()");
+            }
+
+            ByteBuffer writeBuffer = ByteBuffer.wrap(dataToWrite);
+
+            dataFactory.writeInProgress();
+            try {
+                if (SanityManager.DEBUG) {
+                    synchronized(this) {
+                        iosInProgress++;
+                    }
+                }
+
+                writeFull(writeBuffer, ioChannel, pageOffset);
+            } catch (ClosedChannelException ioe) {
+                synchronized(this) {
+                    /* If the write failed because the container has been closed
+                     * for deletion between the start of this method and the
+                     * write, we'll just ignore that, as this container is going
+                     * away anyway.
+                     * This could possibly happen if the Cache is cleaning this
+                     * container while it is dropped - BaseDataFileFactory holds
+                     * an exclusive lock on the container while dropping it to
+                     * avoid other interference.
+                     * See the getCommittedDropState() check at the top of this
+                     * method.
+                     */
+                    if (getCommittedDropState()) {
+                        if (SanityManager.DEBUG) {
+                            SanityManager.DEBUG_PRINT("RAFContainer4",
+                                "Write to a dropped and closed container discarded.");
+                        }
+                        return;
+                    } else {
+                        // This should not happen, better let the exception
+                        // hurt where it's supposed to.
+                        throw ioe;
+                    }
+                }
+            } finally {
+                if (SanityManager.DEBUG) {
+                    synchronized(this) {
+                        iosInProgress--;
+                    }
+                }
+
+                dataFactory.writeFinished();
+            }
+
+            /* Note that the original "try {write} catch IOException { pad file,
+             * write again }" in RAFContainer is removed here, because the
+             * FileChannel Javadoc specifies that the file will be grown to
+             * accommodate the new bytes.
+             */
+
+            if (syncPage) {
+                dataFactory.writeInProgress();
+                try{
+                    if (SanityManager.DEBUG) {
+                        synchronized(this) {
+                            iosInProgress++;
+                        }
+                    }
+                    if (!dataFactory.dataNotSyncedAtAllocation) {
+                        ioChannel.force(false);
+                    }
+                } finally {
+                    if (SanityManager.DEBUG) {
+                        synchronized(this) {
+                            iosInProgress--;
+                        }
+                    }
+                    dataFactory.writeFinished();
+                }
+            } else {
+                synchronized(this) {
+                    needsSync = true;
+                }
+            }
+
+        } else { // iochannel was not initialized, fall back to original method.
+            if(SanityManager.DEBUG) {
+                SanityManager.DEBUG_PRINT("RAFContainer4",
+                        "New style writePage attempted with uninitialized ioChannel");
+            }
+            super.writePage(pageNumber, pageData, syncPage);
+        }
+    }
+
+
+    /**
+     * Attempts to fill buf completely from start until it's full.
+     * <p>
+     * FileChannel has no readFull() method, so we roll our own.
+     */
+    private final void readFull(ByteBuffer dstBuffer,
+                                FileChannel srcChannel,
+                                long position)
+        throws IOException
+    {
+        while(dstBuffer.remaining() > 0) {
+            if( srcChannel.read(dstBuffer, position + dstBuffer.position())
+                    == -1)
+            {
+                throw new EOFException(
+                        "Reached end of file while attempting to read a "
+                        + "whole page.");
+            }
+        }
+    }
+
+    /**
+     * Attempts to write buf completely from start until end, at the given
+     * position in the destination fileChannel.
+     * <p>
+     * FileChannel has no writeFull() method, so we roll our own.
+     */
+    private final void writeFull(ByteBuffer srcBuffer,
+                                 FileChannel dstChannel,
+                                 long position)
+        throws IOException
+    {
+        while(srcBuffer.remaining() > 0) {
+            dstChannel.write(srcBuffer, position + srcBuffer.position());
+        }
+    }
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainer4.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainerFactory.java
URL: http://svn.apache.org/viewvc/db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainerFactory.java?view=auto&rev=447815
==============================================================================
--- db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainerFactory.java (added)
+++ db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainerFactory.java Tue Sep 19 03:58:03 2006
@@ -0,0 +1,110 @@
+/*
+
+   Derby - RAFContainerFactory
+
+   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.store.raw.data;
+
+import org.apache.derby.iapi.services.info.JVMInfo;
+import org.apache.derby.iapi.services.sanity.SanityManager;
+
+import java.lang.reflect.Constructor;
+
+
+/**
+ * Constructs <code>RAFContainer</code> objects suitable for the running JVM.
+ * Java 1.4 and above get <code>RAFContainer4</code> objects capable of
+ * utilizing Java's New IO framework to perform thread-safe concurrent IO to a
+ * container. Java 1.3 gets plain <code>RAFContainer<code> objects that
+ * serialize all IOs to the container.
+ * <p>
+ * Derby's support for Java 1.3 is deprecated as of this writing, and when
+ * support is ultimately removed this class can go away with it as
+ * <code>RAFContainer4</code> is merged back into <code>RAFContainer</code>.
+ */
+class RAFContainerFactory {
+
+    /**
+     * Used to create <code>RAFContainer</code> or <code>RAFContainer4</code
+     * objects as appropriate depending on what JVM we are running in.
+     * <p>
+     * MT: Immutable, initialized by constructor.
+     *
+     * @see newRAFContainer(BaseDataFileFactory factory)
+     */
+    protected final Constructor rafContainerConstructor;
+
+    /**
+     * Determines what JVM we're running in and loads the appropriate
+     * <code>RAFContainer</code> class, and stores its Constructor in
+     * rafContainerConstructor for use in
+     * <code>newRafContainer(BaseDataFileFactory factory)</code>.
+     */
+    public RAFContainerFactory() {
+        Constructor foundConstructor = null;
+        try {
+            Class containerClass;
+            Class factoryClass = BaseDataFileFactory.class;
+
+            if( JVMInfo.JDK_ID >= JVMInfo.J2SE_14) {
+                containerClass = Class.forName(
+                        "org.apache.derby.impl.store.raw.data.RAFContainer4");
+            } else {
+                containerClass = RAFContainer.class;
+            }
+
+            foundConstructor = containerClass.getDeclaredConstructor(
+                    new Class[] {factoryClass});
+        } catch (Exception e) {
+            if(SanityManager.DEBUG) {
+                SanityManager.DEBUG_PRINT("RAFContainerFactory",
+                        "Caught exception when setting up rafContainerConstructor");
+            }
+            /*
+             * If there's a problem we'll back away from this trick.
+             * newRAFContainer() checks for null and uses a regular
+             * reflectionless <code>return new RAFContainer(this)</code> if
+             * that's the case.
+             */
+        } finally { // Ensure that rafContainerConstructor is defined
+            rafContainerConstructor = foundConstructor;
+        }
+    }
+
+    /**
+     * Produces a <code>RAFContainer</code> object appropriate for this JVM.
+     */
+    public RAFContainer newRAFContainer(BaseDataFileFactory factory) {
+        if(rafContainerConstructor != null) {
+            try {
+                return (RAFContainer) rafContainerConstructor.newInstance(
+                            new BaseDataFileFactory[] {factory});
+            } catch (Exception e) {
+                if(SanityManager.DEBUG) {
+                    SanityManager.DEBUG_PRINT("RAFContainerFactory",
+                            "Caught exception when attempting to create RAFContainer object");
+                }
+                // Falls through and constructs old-style RAFContainer instead.
+            }
+        }
+        /* If rafContainerConstructor is null, the static initializer may
+         * have failed to load the JVM 1.4 class or its constructor */
+        return new RAFContainer(factory);
+    }
+}

Propchange: db/derby/code/trunk/java/engine/org/apache/derby/impl/store/raw/data/RAFContainerFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native