You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by iv...@apache.org on 2012/03/20 14:09:08 UTC

svn commit: r1302870 - in /zookeeper/bookkeeper/trunk: ./ bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/

Author: ivank
Date: Tue Mar 20 13:09:07 2012
New Revision: 1302870

URL: http://svn.apache.org/viewvc?rev=1302870&view=rev
Log:
BOOKKEEPER-175: Bookie code is very coupled (ivank)

Added:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactory.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java
      - copied, changed from r1302851, zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java
Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1302870&r1=1302869&r2=1302870&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Tue Mar 20 13:09:07 2012
@@ -92,6 +92,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-187: Create well defined interface for LedgerCache (ivank)
 
+        BOOKKEEPER-175: Bookie code is very coupled (ivank)
+
       hedwig-server/
 
         BOOKKEEPER-77: Add a console client for hedwig (Sijie Guo via ivank)

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java?rev=1302870&r1=1302869&r2=1302870&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java Tue Mar 20 13:09:07 2012
@@ -39,6 +39,7 @@ import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Map;
 import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
@@ -67,7 +68,6 @@ import org.apache.zookeeper.ZooDefs.Ids;
  */
 
 public class Bookie extends Thread {
-    HashMap<Long, LedgerDescriptor> ledgers = new HashMap<Long, LedgerDescriptor>();
     static Logger LOG = LoggerFactory.getLogger(Bookie.class);
 
     final static long MB = 1024 * 1024L;
@@ -84,6 +84,7 @@ public class Bookie extends Thread {
 
     final SyncThread syncThread;
     final LedgerManager ledgerManager;
+    final HandleFactory handles;
 
     static final long METAENTRY_ID_LEDGER_KEY = -0x1000;
 
@@ -106,6 +107,8 @@ public class Bookie extends Thread {
     BookieBean jmxBookieBean;
     LedgerCacheBean jmxLedgerCacheBean;
 
+    Map<Long, byte[]> masterKeyCache = Collections.synchronizedMap(new HashMap<Long, byte[]>());
+
     public static class NoLedgerException extends IOException {
         private static final long serialVersionUID = 1L;
         private long ledgerId;
@@ -402,8 +405,9 @@ public class Bookie extends Thread {
         entryLogger = new EntryLogger(conf);
         ledgerCache = new LedgerCacheImpl(conf, ledgerManager);
         gcThread = new GarbageCollectorThread(conf, this.zk, ledgerCache, entryLogger,
-                                              ledgerManager,
-                                              new EntryLogCompactionScanner());
+                ledgerManager, new EntryLogCompactionScanner());
+        handles = new HandleFactoryImpl(entryLogger, ledgerCache);
+
         // replay journals
         readJournal();
     }
@@ -471,29 +475,29 @@ public class Bookie extends Thread {
                 recBuff.flip();
                 long ledgerId = recBuff.getLong();
                 long entryId = recBuff.getLong();
-                LedgerDescriptor handle;
                 try {
-                    handle = getHandle(ledgerId);
-                    LOG.debug("Relay journal - ledger id : {}, entry id : {}.", ledgerId, entryId);
+                    LOG.debug("Replay journal - ledger id : {}, entry id : {}.", ledgerId, entryId);
                     if (entryId == METAENTRY_ID_LEDGER_KEY) {
                         if (recLog.getFormatVersion() >= 3) {
                             int masterKeyLen = recBuff.getInt();
                             byte[] masterKey = new byte[masterKeyLen];
+
                             recBuff.get(masterKey);
-                            handle.checkAccess(masterKey);
-                            putHandle(handle);
+                            masterKeyCache.put(ledgerId, masterKey);
                         } else {
                             throw new IOException("Invalid journal. Contains journalKey "
                                     + " but layout version (" + recLog.getFormatVersion()
                                     + ") is too old to hold this");
                         }
                     } else {
-                        try {
-                            recBuff.rewind();
-                            handle.addEntry(recBuff);
-                        } finally {
-                            putHandle(handle);
+                        byte[] key = masterKeyCache.get(ledgerId);
+                        if (key == null) {
+                            key = ledgerCache.readMasterKey(ledgerId);
                         }
+                        LedgerDescriptor handle = handles.getHandle(ledgerId, key);
+
+                        recBuff.rewind();
+                        handle.addEntry(recBuff);
                     }
                 } catch (NoLedgerException nsle) {
                     LOG.debug("Skip replaying entries of ledger {} since it was deleted.", ledgerId);
@@ -697,46 +701,6 @@ public class Bookie extends Thread {
         }
         return total;
     }
-    private void putHandle(LedgerDescriptor handle) {
-        synchronized (ledgers) {
-            handle.decRef();
-        }
-    }
-
-    private LedgerDescriptor getHandle(long ledgerId, boolean readonly, byte[] masterKey) 
-            throws IOException, BookieException {
-        LedgerDescriptor handle = null;
-        synchronized (ledgers) {
-            handle = ledgers.get(ledgerId);
-            if (handle == null) {
-                if (readonly) {
-                    throw new NoLedgerException(ledgerId);
-                }
-                handle = createHandle(ledgerId);
-                ledgers.put(ledgerId, handle);
-            }
-            handle.checkAccess(masterKey);
-            handle.incRef();
-        }
-        return handle;
-    }
-
-    private LedgerDescriptor getHandle(long ledgerId) throws IOException {
-        LedgerDescriptor handle = null;
-        synchronized (ledgers) {
-            handle = ledgers.get(ledgerId);
-            if (handle == null) {
-                handle = createHandle(ledgerId);
-                ledgers.put(ledgerId, handle);
-            }
-            handle.incRef();
-        }
-        return handle;
-    }
-
-    private LedgerDescriptor createHandle(long ledgerId) throws IOException {
-        return new LedgerDescriptor(ledgerId, entryLogger, ledgerCache);
-    }
 
     static class QueueEntry {
         QueueEntry(ByteBuffer entry, long ledgerId, long entryId,
@@ -970,9 +934,7 @@ public class Bookie extends Thread {
                 this.interrupt();
                 this.join();
                 syncThread.shutdown();
-                for(LedgerDescriptor d: ledgers.values()) {
-                    d.close();
-                }
+
                 // Shutdown the EntryLogger which has the GarbageCollector Thread running
                 entryLogger.shutdown();
                 // close Ledger Manager
@@ -996,8 +958,8 @@ public class Bookie extends Thread {
     private LedgerDescriptor getLedgerForEntry(ByteBuffer entry, byte[] masterKey) 
             throws IOException, BookieException {
         long ledgerId = entry.getLong();
-        LedgerDescriptor l = getHandle(ledgerId, false, masterKey);
-        if (!l.isMasterKeyPersisted()) {
+        LedgerDescriptor l = handles.getHandle(ledgerId, masterKey);
+        if (!masterKeyCache.containsKey(ledgerId)) {
             // new handle, we should add the key to journal ensure we can rebuild
             ByteBuffer bb = ByteBuffer.allocate(8 + 8 + 4 + masterKey.length);
             bb.putLong(ledgerId);
@@ -1015,19 +977,16 @@ public class Bookie extends Thread {
                                              // do nothing
                                          }
                                      }, null));
-            l.setMasterKeyPersisted();
+            masterKeyCache.put(ledgerId, masterKey);
         }
         return l;
     }
 
     protected void addEntryByLedgerId(long ledgerId, ByteBuffer entry)
         throws IOException, BookieException {
-        LedgerDescriptor handle = getHandle(ledgerId);
-        try {
-            handle.addEntry(entry);
-        } finally {
-            putHandle(handle);
-        }
+        byte[] key = ledgerCache.readMasterKey(ledgerId);
+        LedgerDescriptor handle = handles.getHandle(ledgerId, key);
+        handle.addEntry(entry);
     }
 
     /**
@@ -1056,11 +1015,7 @@ public class Bookie extends Thread {
             throws IOException, BookieException {
         LedgerDescriptor handle = getLedgerForEntry(entry, masterKey);
         synchronized (handle) {
-            try {
-                addEntryInternal(handle, entry, cb, ctx);
-            } finally {
-                putHandle(handle);
-            }
+            addEntryInternal(handle, entry, cb, ctx);
         }
     }
     
@@ -1072,15 +1027,11 @@ public class Bookie extends Thread {
             throws IOException, BookieException {
         LedgerDescriptor handle = getLedgerForEntry(entry, masterKey);
         synchronized (handle) {
-            try {
-                if (handle.isFenced()) {
-                    throw BookieException.create(BookieException.Code.LedgerFencedException);
-                }
-                
-                addEntryInternal(handle, entry, cb, ctx);
-            } finally {
-                putHandle(handle);
+            if (handle.isFenced()) {
+                throw BookieException.create(BookieException.Code.LedgerFencedException);
             }
+
+            addEntryInternal(handle, entry, cb, ctx);
         }
     }
 
@@ -1092,22 +1043,24 @@ public class Bookie extends Thread {
      * never be unfenced. Fencing a fenced ledger has no effect.
      */
     public void fenceLedger(long ledgerId) throws IOException {
-        LedgerDescriptor handle = getHandle(ledgerId);
-        synchronized (handle) {
-            handle.setFenced();
+        try {
+            byte[] key = ledgerCache.readMasterKey(ledgerId);
+            LedgerDescriptor handle = handles.getHandle(ledgerId, key);
+            synchronized (handle) {
+                handle.setFenced();
+            }
+        } catch (BookieException e) {
+            throw new IOException("Error fencing", e);
         }
     }
 
-    public ByteBuffer readEntry(long ledgerId, long entryId) throws IOException {
-        LedgerDescriptor handle = getHandle(ledgerId);
-        try {
-            if (LOG.isTraceEnabled()) {
-                LOG.trace("Reading " + entryId + "@" + ledgerId);
-            }
-            return handle.readEntry(entryId);
-        } finally {
-            putHandle(handle);
+    public ByteBuffer readEntry(long ledgerId, long entryId)
+            throws IOException, NoLedgerException {
+        LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("Reading " + entryId + "@" + ledgerId);
         }
+        return handle.readEntry(entryId);
     }
 
     // The rest of the code is test stuff

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactory.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactory.java?rev=1302870&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactory.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactory.java Tue Mar 20 13:09:07 2012
@@ -0,0 +1,32 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import java.io.IOException;
+
+interface HandleFactory {
+    LedgerDescriptor getHandle(long ledgerId, byte[] masterKey)
+            throws IOException, BookieException;
+
+    LedgerDescriptor getReadOnlyHandle(long ledgerId)
+            throws IOException, Bookie.NoLedgerException;
+}
\ No newline at end of file

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java?rev=1302870&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/HandleFactoryImpl.java Tue Mar 20 13:09:07 2012
@@ -0,0 +1,69 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+class HandleFactoryImpl implements HandleFactory {
+    HashMap<Long, LedgerDescriptor> ledgers = new HashMap<Long, LedgerDescriptor>();
+    HashMap<Long, LedgerDescriptor> readOnlyLedgers
+        = new HashMap<Long, LedgerDescriptor>();
+
+    final EntryLogger entryLogger;
+    final LedgerCache ledgerCache;
+
+    HandleFactoryImpl(EntryLogger entryLogger, LedgerCache ledgerCache) {
+        this.entryLogger = entryLogger;
+        this.ledgerCache = ledgerCache;
+    }
+
+    @Override
+    public LedgerDescriptor getHandle(long ledgerId, byte[] masterKey)
+            throws IOException, BookieException {
+        LedgerDescriptor handle = null;
+        synchronized (ledgers) {
+            handle = ledgers.get(ledgerId);
+            if (handle == null) {
+                handle = LedgerDescriptor.create(masterKey, ledgerId,
+                                                 entryLogger, ledgerCache);
+                ledgers.put(ledgerId, handle);
+            }
+            handle.checkAccess(masterKey);
+        }
+        return handle;
+    }
+
+    @Override
+    public LedgerDescriptor getReadOnlyHandle(long ledgerId)
+            throws IOException, Bookie.NoLedgerException {
+        LedgerDescriptor handle = null;
+        synchronized (ledgers) {
+            handle = readOnlyLedgers.get(ledgerId);
+            if (handle == null) {
+                handle = LedgerDescriptor.createReadOnly(ledgerId, entryLogger, ledgerCache);
+                readOnlyLedgers.put(ledgerId, handle);
+            }
+        }
+        return handle;
+    }
+}
\ No newline at end of file

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java?rev=1302870&r1=1302869&r2=1302870&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java Tue Mar 20 13:09:07 2012
@@ -28,125 +28,37 @@ import java.util.Arrays;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-
 /**
  * Implements a ledger inside a bookie. In particular, it implements operations
  * to write entries to a ledger and read entries from a ledger.
- *
  */
-public class LedgerDescriptor {
-    final static Logger LOG = LoggerFactory.getLogger(LedgerDescriptor.class);
-    LedgerCacheImpl ledgerCache;
-    LedgerDescriptor(long ledgerId, EntryLogger entryLogger, LedgerCache ledgerCache) {
-        this.ledgerId = ledgerId;
-        this.entryLogger = entryLogger;
-        // This cast is only here until ledgerDescriptor changes go in to make it
-        // unnecessary
-        this.ledgerCache = (LedgerCacheImpl)ledgerCache;
+public abstract class LedgerDescriptor {
+    static LedgerDescriptor create(byte[] masterKey,
+                                   long ledgerId,
+                                   EntryLogger entryLogger,
+                                   LedgerCache ledgerCache) throws IOException {
+        LedgerDescriptor ledger = new LedgerDescriptorImpl(masterKey, ledgerId, entryLogger, ledgerCache);
+        ledgerCache.setMasterKey(ledgerId, masterKey);
+        return ledger;
     }
 
-    private byte[] masterKey = null;
-    volatile private boolean fenced = false;
-    private boolean masterKeyPersisted = false;
-
-    synchronized boolean isMasterKeyPersisted() {
-        if (masterKeyPersisted) {
-            return true;
-        }
-
-        try {
-            FileInfo fi = ledgerCache.getFileInfo(ledgerId, masterKey);
-            fi.readHeader();
-            masterKeyPersisted = true;
-            return true;
-        } catch (IOException ioe) {
-            return false;
+    static LedgerDescriptor createReadOnly(long ledgerId,
+                                           EntryLogger entryLogger,
+                                           LedgerCache ledgerCache)
+            throws IOException, Bookie.NoLedgerException {
+        if (!ledgerCache.ledgerExists(ledgerId)) {
+            throw new Bookie.NoLedgerException(ledgerId);
         }
+        return new LedgerDescriptorReadOnlyImpl(ledgerId, entryLogger, ledgerCache);
     }
 
-    void setMasterKeyPersisted() {
-        masterKeyPersisted = true;
-    }
+    abstract void checkAccess(byte masterKey[]) throws BookieException, IOException;
 
-    void checkAccess(byte masterKey[]) throws BookieException, IOException {
-        if (this.masterKey == null) {
-            FileInfo fi = ledgerCache.getFileInfo(ledgerId, masterKey);
-            try {
-                if (fi == null) {
-                    throw new IOException(ledgerId + " does not exist");
-                }
-                this.masterKey = fi.getMasterKey();
-            } finally {
-                fi.release();
-            }
-        }
-        if (!Arrays.equals(this.masterKey, masterKey)) {
-            throw BookieException.create(BookieException.Code.UnauthorizedAccessException);
-        }
-    } 
+    abstract long getLedgerId();
 
-    private long ledgerId;
-    public long getLedgerId() {
-        return ledgerId;
-    }
+    abstract void setFenced() throws IOException;
+    abstract boolean isFenced();
 
-    EntryLogger entryLogger;
-    private int refCnt;
-    synchronized public void incRef() {
-        refCnt++;
-    }
-    synchronized public void decRef() {
-        refCnt--;
-    }
-    synchronized public int getRefCnt() {
-        return refCnt;
-    }
-    
-    void setFenced() {
-        fenced = true;
-    }
-    
-    boolean isFenced() {
-        return fenced;
-    }
-
-    long addEntry(ByteBuffer entry) throws IOException {
-        long ledgerId = entry.getLong();
-
-        if (ledgerId != this.ledgerId) {
-            throw new IOException("Entry for ledger " + ledgerId + " was sent to " + this.ledgerId);
-        }
-        long entryId = entry.getLong();
-        entry.rewind();
-
-        /*
-         * Log the entry
-         */
-        long pos = entryLogger.addEntry(ledgerId, entry);
-
-
-        /*
-         * Set offset of entry id to be the current ledger position
-         */
-        ledgerCache.putEntryOffset(ledgerId, entryId, pos);
-        return entryId;
-    }
-    ByteBuffer readEntry(long entryId) throws IOException {
-        long offset;
-        /*
-         * If entryId is -1, then return the last written.
-         */
-        if (entryId == -1) {
-            entryId = ledgerCache.getLastEntry(ledgerId);
-        }
-
-        offset = ledgerCache.getEntryOffset(ledgerId, entryId);
-        if (offset == 0) {
-            throw new Bookie.NoEntryException(ledgerId, entryId);
-        }
-        return ByteBuffer.wrap(entryLogger.readEntry(ledgerId, entryId, offset));
-    }
-    void close() {
-    }
+    abstract long addEntry(ByteBuffer entry) throws IOException;
+    abstract ByteBuffer readEntry(long entryId) throws IOException;
 }

Copied: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java (from r1302851, zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java)
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java?p2=zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java&p1=zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java&r1=1302851&r2=1302870&rev=1302870&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java Tue Mar 20 13:09:07 2012
@@ -28,89 +28,50 @@ import java.util.Arrays;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-
-
 /**
  * Implements a ledger inside a bookie. In particular, it implements operations
  * to write entries to a ledger and read entries from a ledger.
  *
  */
-public class LedgerDescriptor {
+public class LedgerDescriptorImpl extends LedgerDescriptor {
     final static Logger LOG = LoggerFactory.getLogger(LedgerDescriptor.class);
-    LedgerCacheImpl ledgerCache;
-    LedgerDescriptor(long ledgerId, EntryLogger entryLogger, LedgerCache ledgerCache) {
-        this.ledgerId = ledgerId;
-        this.entryLogger = entryLogger;
-        // This cast is only here until ledgerDescriptor changes go in to make it
-        // unnecessary
-        this.ledgerCache = (LedgerCacheImpl)ledgerCache;
-    }
+    LedgerCache ledgerCache;
+    private long ledgerId;
+    EntryLogger entryLogger;
 
-    private byte[] masterKey = null;
     volatile private boolean fenced = false;
-    private boolean masterKeyPersisted = false;
-
-    synchronized boolean isMasterKeyPersisted() {
-        if (masterKeyPersisted) {
-            return true;
-        }
-
-        try {
-            FileInfo fi = ledgerCache.getFileInfo(ledgerId, masterKey);
-            fi.readHeader();
-            masterKeyPersisted = true;
-            return true;
-        } catch (IOException ioe) {
-            return false;
-        }
-    }
+    final byte[] masterKey;
 
-    void setMasterKeyPersisted() {
-        masterKeyPersisted = true;
+    LedgerDescriptorImpl(byte[] masterKey, long ledgerId, EntryLogger entryLogger, LedgerCache ledgerCache) {
+        this.masterKey = masterKey;
+        this.ledgerId = ledgerId;
+        this.entryLogger = entryLogger;
+        this.ledgerCache = ledgerCache;
     }
 
+    @Override
     void checkAccess(byte masterKey[]) throws BookieException, IOException {
-        if (this.masterKey == null) {
-            FileInfo fi = ledgerCache.getFileInfo(ledgerId, masterKey);
-            try {
-                if (fi == null) {
-                    throw new IOException(ledgerId + " does not exist");
-                }
-                this.masterKey = fi.getMasterKey();
-            } finally {
-                fi.release();
-            }
-        }
         if (!Arrays.equals(this.masterKey, masterKey)) {
             throw BookieException.create(BookieException.Code.UnauthorizedAccessException);
         }
-    } 
+    }
 
-    private long ledgerId;
+    @Override
     public long getLedgerId() {
         return ledgerId;
     }
 
-    EntryLogger entryLogger;
-    private int refCnt;
-    synchronized public void incRef() {
-        refCnt++;
-    }
-    synchronized public void decRef() {
-        refCnt--;
-    }
-    synchronized public int getRefCnt() {
-        return refCnt;
-    }
-    
-    void setFenced() {
+    @Override
+    void setFenced() throws IOException {
         fenced = true;
     }
-    
+
+    @Override
     boolean isFenced() {
         return fenced;
     }
 
+    @Override
     long addEntry(ByteBuffer entry) throws IOException {
         long ledgerId = entry.getLong();
 
@@ -132,6 +93,8 @@ public class LedgerDescriptor {
         ledgerCache.putEntryOffset(ledgerId, entryId, pos);
         return entryId;
     }
+
+    @Override
     ByteBuffer readEntry(long entryId) throws IOException {
         long offset;
         /*
@@ -147,6 +110,4 @@ public class LedgerDescriptor {
         }
         return ByteBuffer.wrap(entryLogger.readEntry(ledgerId, entryId, offset));
     }
-    void close() {
-    }
 }

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java?rev=1302870&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorReadOnlyImpl.java Tue Mar 20 13:09:07 2012
@@ -0,0 +1,53 @@
+/*
+ *
+ * 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.bookkeeper.bookie;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+/**
+ * Implements a ledger inside a bookie. In particular, it implements operations
+ * to write entries to a ledger and read entries from a ledger.
+ */
+public class LedgerDescriptorReadOnlyImpl extends LedgerDescriptorImpl {
+    LedgerDescriptorReadOnlyImpl(long ledgerId, EntryLogger entryLogger,
+            LedgerCache ledgerCache) {
+        super(null, ledgerId, entryLogger, ledgerCache);
+    }
+
+    @Override
+    void setFenced() throws IOException {
+        assert false;
+        throw new IOException("Invalid action on read only descriptor");
+    }
+
+    @Override
+    long addEntry(ByteBuffer entry) throws IOException {
+        assert false;
+        throw new IOException("Invalid action on read only descriptor");
+    }
+
+    @Override
+    void checkAccess(byte masterKey[]) throws BookieException, IOException {
+        assert false;
+        throw new IOException("Invalid action on read only descriptor");
+    }
+}