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/11/09 17:13:12 UTC

svn commit: r1407520 [1/2] - in /zookeeper/bookkeeper/trunk: ./ bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/ bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/ bookkeeper-server/src/test/java/org/apache/bookkeeper...

Author: ivank
Date: Fri Nov  9 16:13:10 2012
New Revision: 1407520

URL: http://svn.apache.org/viewvc?rev=1407520&view=rev
Log:
BOOKKEEPER-204: Provide a MetaStore interface, and a mock implementation. (Jiannan Wang via ivank)

Added:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MSException.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetaStore.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCallback.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCursor.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreException.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreScannableTable.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTable.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTableItem.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetaStore.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreCursor.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreTable.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreScannableTableAsyncToSyncConverter.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreTableAsyncToSyncConverter.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/TestMetaStore.java
Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1407520&r1=1407519&r2=1407520&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Fri Nov  9 16:13:10 2012
@@ -180,6 +180,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-444: Refactor pending read op to make speculative reads possible (ivank)
 
+        BOOKKEEPER-204: Provide a MetaStore interface, and a mock implementation. (Jiannan Wang via ivank)
+
       hedwig-server:
 
         BOOKKEEPER-250: Need a ledger manager like interface to manage metadata operations in Hedwig (sijie via ivank)

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MSException.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MSException.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MSException.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MSException.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,214 @@
+/**
+ * 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.metastore;
+
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.Map;
+
+@SuppressWarnings("serial")
+public abstract class MSException extends Exception {
+
+    /**
+     * return codes
+     */
+    public static enum Code {
+        OK (0, "OK"),
+        BadVersion (-1, "Version conflict"),
+        NoKey (-2, "Key does not exist"),
+        KeyExists (-3, "Key exists"),
+        NoEntries (-4, "No entries found"),
+
+        InterruptedException (-100, "Operation interrupted"),
+        IllegalOp (-101, "Illegal operation"),
+        ServiceDown (-102, "Metadata service is down"),
+        OperationFailure(-103, "Operaion failed on metadata storage server side");
+
+        private static final Map<Integer, Code> codes
+            = new HashMap<Integer, Code>();
+
+        static {
+            for (Code c : EnumSet.allOf(Code.class)) {
+                codes.put(c.code, c);
+            }
+        }
+
+        private final int code;
+        private final String description;
+
+        private Code(int code, String description) {
+            this.code = code;
+            this.description = description;
+        }
+
+        /**
+         * Get the int value for a particular Code.
+         *
+         * @return error code as integer
+         */
+        public int getCode() {
+            return code;
+        }
+
+        /**
+         * Get the description for a particular Code.
+         *
+         * @return error description
+         */
+        public String getDescription() {
+            return description;
+        }
+
+        /**
+         * Get the Code value for a particular integer error code.
+         *
+         * @param code int error code
+         * @return Code value corresponding to specified int code, or null.
+         */
+        public static Code get(int code) {
+            return codes.get(code);
+        }
+    }
+
+    private final Code code;
+
+    MSException(Code code, String errMsg) {
+        super(code.getDescription() + " : " + errMsg);
+        this.code = code;
+    }
+
+    MSException(Code code, String errMsg, Throwable cause) {
+        super(code.getDescription() + " : " + errMsg, cause);
+        this.code = code;
+    }
+
+    public Code getCode() {
+        return this.code;
+    }
+
+    public static MSException create(Code code) {
+        return create(code, "", null);
+    }
+
+    public static MSException create(Code code, String errMsg) {
+        return create(code, errMsg, null);
+    }
+
+    public static MSException create(Code code, String errMsg, Throwable cause) {
+        switch (code) {
+            case BadVersion:
+                return new BadVersionException(errMsg, cause);
+            case NoKey:
+                return new NoKeyException(errMsg, cause);
+            case KeyExists:
+                return new KeyExistsException(errMsg, cause);
+            case InterruptedException:
+                return new MSInterruptedException(errMsg, cause);
+            case IllegalOp:
+                return new IllegalOpException(errMsg, cause);
+            case ServiceDown:
+                return new ServiceDownException(errMsg, cause);
+            case OperationFailure:
+                return new OperationFailureException(errMsg, cause);
+            case OK:
+            default:
+                throw new IllegalArgumentException("Invalid exception code");
+        }
+    }
+
+    public static class BadVersionException extends MSException {
+        public BadVersionException(String errMsg) {
+            super(Code.BadVersion, errMsg);
+        }
+
+        public BadVersionException(String errMsg, Throwable cause) {
+            super(Code.BadVersion, errMsg, cause);
+        }
+    }
+
+    public static class NoKeyException extends MSException {
+        public NoKeyException(String errMsg) {
+            super(Code.NoKey, errMsg);
+        }
+
+        public NoKeyException(String errMsg, Throwable cause) {
+            super(Code.NoKey, errMsg, cause);
+        }
+    }
+
+    // Exception would be thrown in a cursor if no entries found
+    public static class NoEntriesException extends MSException {
+        public NoEntriesException(String errMsg) {
+            super(Code.NoEntries, errMsg);
+        }
+
+        public NoEntriesException(String errMsg, Throwable cause) {
+            super(Code.NoEntries, errMsg, cause);
+        }
+    }
+
+    public static class KeyExistsException extends MSException {
+        public KeyExistsException(String errMsg) {
+            super(Code.KeyExists, errMsg);
+        }
+
+        public KeyExistsException(String errMsg, Throwable cause) {
+            super(Code.KeyExists, errMsg, cause);
+        }
+    }
+
+    public static class MSInterruptedException extends MSException {
+        public MSInterruptedException(String errMsg) {
+            super(Code.InterruptedException, errMsg);
+        }
+
+        public MSInterruptedException(String errMsg, Throwable cause) {
+            super(Code.InterruptedException, errMsg, cause);
+        }
+    }
+
+    public static class IllegalOpException extends MSException {
+        public IllegalOpException(String errMsg) {
+            super(Code.IllegalOp, errMsg);
+        }
+
+        public IllegalOpException(String errMsg, Throwable cause) {
+            super(Code.IllegalOp, errMsg, cause);
+        }
+    }
+
+    public static class ServiceDownException extends MSException {
+        public ServiceDownException(String errMsg) {
+            super(Code.ServiceDown, errMsg);
+        }
+
+        public ServiceDownException(String errMsg, Throwable cause) {
+            super(Code.ServiceDown, errMsg, cause);
+        }
+    }
+
+    public static class OperationFailureException extends MSException {
+        public OperationFailureException(String errMsg) {
+            super(Code.OperationFailure, errMsg);
+        }
+
+        public OperationFailureException(String errMsg, Throwable cause) {
+            super(Code.OperationFailure, errMsg, cause);
+        }
+    }
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetaStore.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetaStore.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetaStore.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetaStore.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,79 @@
+/**
+ * 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.metastore;
+
+import org.apache.commons.configuration.Configuration;
+
+/**
+ * Metadata Store Interface.
+ */
+public interface MetaStore {
+    /**
+     * Return the name of the plugin.
+     *
+     * @return the plugin name.
+     */
+    public String getName();
+
+    /**
+     * Get the plugin verison.
+     *
+     * @return the plugin version.
+     */
+    public int getVersion();
+
+    /**
+     * Initialize the meta store.
+     *
+     * @param config
+     *          Configuration object passed to metastore
+     * @param msVersion
+     *          Version to initialize the metastore
+     * @throws MetastoreException when failed to initialize
+     */
+    public void init(Configuration config, int msVersion)
+    throws MetastoreException;
+
+    /**
+     * Close the meta store.
+     */
+    public void close();
+
+    /**
+     * Create a metastore table.
+     *
+     * @param name
+     *          Table name.
+     * @return a metastore table
+     * @throws MetastoreException when failed to create the metastore table.
+     */
+    public MetastoreTable createTable(String name)
+    throws MetastoreException;
+
+    /**
+     * Create a scannable metastore table.
+     *
+     * @param name
+     *          Table name.
+     * @return a metastore scannable table
+     * @throws MetastoreException when failed to create the metastore table.
+     */
+    public MetastoreScannableTable createScannableTable(String name)
+    throws MetastoreException;
+
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCallback.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCallback.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCallback.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCallback.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,25 @@
+/**
+ * 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.metastore;
+
+public interface MetastoreCallback<T> {
+    /**
+     * @see MSException.Code
+     */
+    public void complete(int rc, T value, Object ctx);
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCursor.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCursor.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCursor.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreCursor.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,84 @@
+/**
+ * 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.metastore;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Iterator;
+
+public interface MetastoreCursor extends Closeable {
+
+    public static MetastoreCursor EMPTY_CURSOR = new MetastoreCursor() {
+        @Override
+        public boolean hasMoreEntries() {
+            return false;
+        }
+
+        @Override
+        public Iterator<MetastoreTableItem> readEntries(int numEntries)
+        throws MSException {
+            throw new MSException.NoEntriesException("No entries left in the cursor.");
+        }
+
+        @Override
+        public void asyncReadEntries(int numEntries, ReadEntriesCallback callback, Object ctx) {
+            callback.complete(MSException.Code.NoEntries.getCode(), null, ctx);
+        }
+
+        @Override
+        public void close() throws IOException {
+            // do nothing
+        }
+    };
+
+    public static interface ReadEntriesCallback extends
+        MetastoreCallback<Iterator<MetastoreTableItem>> {
+    }
+
+    /**
+     * Is there any entries left in the cursor to read.
+     *
+     * @return true if there is entries left, false otherwise.
+     */
+    public boolean hasMoreEntries();
+
+    /**
+     * Read entries from the cursor, up to the specified <code>numEntries</code>.
+     * The returned list can be smaller.
+     *
+     * @param numEntries
+     *            maximum number of entries to read
+     * @return the iterator of returned entries.
+     * @throws MSException when failed to read entries from the cursor.
+     */
+    public Iterator<MetastoreTableItem> readEntries(int numEntries)
+    throws MSException;
+
+    /**
+     * Asynchronously read entries from the cursor, up to the specified <code>numEntries</code>.
+     *
+     * @see #readEntries(int)
+     * @param numEntries
+     *            maximum number of entries to read
+     * @param callback
+     *            callback object
+     * @param ctx
+     *            opaque context
+     */
+    public void asyncReadEntries(int numEntries, ReadEntriesCallback callback, Object ctx);
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreException.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreException.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreException.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreException.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,34 @@
+/**
+ * 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.metastore;
+
+@SuppressWarnings("serial")
+public class MetastoreException extends Exception {
+
+    public MetastoreException(String message) {
+        super(message);
+    }
+
+    public MetastoreException(String message, Throwable t) {
+        super(message, t);
+    }
+
+    public MetastoreException(Throwable t) {
+        super(t);
+    }
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreFactory.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,33 @@
+/**
+ * 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.metastore;
+
+import org.apache.bookkeeper.util.ReflectionUtils;
+
+public class MetastoreFactory {
+
+    public static MetaStore createMetaStore(String name)
+    throws MetastoreException {
+        try {
+            return ReflectionUtils.newInstance(name, MetaStore.class);
+        } catch (Throwable t) {
+            throw new MetastoreException("Failed to instantiate metastore : " + name);
+        }
+    }
+
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreScannableTable.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreScannableTable.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreScannableTable.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreScannableTable.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,104 @@
+/**
+ * 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.metastore;
+
+import java.util.Set;
+
+public interface MetastoreScannableTable extends MetastoreTable {
+
+    // Used by cursor, etc when they want to start at the beginning of a table
+    public static final String EMPTY_START_KEY = null;
+    // Last row in a table.
+    public static final String EMPTY_END_KEY = null;
+    // the order to loop over a table
+    public static enum Order {
+        ASC,
+        DESC
+    }
+
+    /**
+     * Open a cursor to loop over the entries belonging to a key range,
+     * which returns all fields for each entry.
+     *
+     * <p>
+     * Return Code:<br/>
+     * {@link MSException.Code.OK}: an opened cursor<br/>
+     * {@link MSException.Code.IllegalOp}/{@link MSException.Code.ServiceDown}:
+     * other issues
+     * </p>
+     *
+     * @param firstKey
+     *            Key to start scanning. If it is {@link EMPTY_START_KEY}, it starts
+     *            from first key (inclusive).
+     * @param firstInclusive
+     *            true if firstKey is to be included in the returned view.
+     * @param lastKey
+     *            Key to stop scanning. If it is {@link EMPTY_END_KEY}, scan ends at
+     *            the lastKey of the table (inclusive).
+     * @param lastInclusive
+     *            true if lastKey is to be included in the returned view.
+     * @param order
+     *            the order to loop over the entries
+     * @param cb
+     *            Callback to return an opened cursor.
+     * @param ctx
+     *            Callback context
+     */
+    public void openCursor(String firstKey, boolean firstInclusive,
+                           String lastKey, boolean lastInclusive,
+                           Order order,
+                           MetastoreCallback<MetastoreCursor> cb,
+                           Object ctx);
+
+    /**
+     * Open a cursor to loop over the entries belonging to a key range,
+     * which returns the specified <code>fields</code> for each entry.
+     *
+     * <p>
+     * Return Code:<br/>
+     * {@link MSException.Code.OK}: an opened cursor<br/>
+     * {@link MSException.Code.IllegalOp}/{@link MSException.Code.ServiceDown}:
+     * other issues
+     * </p>
+     *
+     * @param firstKey
+     *            Key to start scanning. If it is {@link EMPTY_START_KEY}, it starts
+     *            from first key (inclusive).
+     * @param firstInclusive
+     *            true if firstKey is to be included in the returned view.
+     * @param lastKey
+     *            Key to stop scanning. If it is {@link EMPTY_END_KEY}, scan ends at
+     *            the lastKey of the table (inclusive).
+     * @param lastInclusive
+     *            true if lastKey is to be included in the returned view.
+     * @param order
+     *            the order to loop over the entries
+     * @param fields
+     *            Fields to select
+     * @param cb
+     *            Callback to return an opened cursor.
+     * @param ctx
+     *            Callback context
+     */
+    public void openCursor(String firstKey, boolean firstInclusive,
+                           String lastKey, boolean lastInclusive,
+                           Order order, Set<String> fields,
+                           MetastoreCallback<MetastoreCursor> cb,
+                           Object ctx);
+
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTable.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTable.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTable.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTable.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,172 @@
+/**
+ * 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.metastore;
+
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+
+public interface MetastoreTable {
+
+    // select all fields when reading or scanning entries
+    public static final Set<String> ALL_FIELDS = null;
+    // select non fields to return when reading/scanning entries
+    public static final Set<String> NON_FIELDS = new HashSet<String>();
+
+    /**
+     * Get table name.
+     *
+     * @return table name
+     */
+    public String getName();
+
+    /**
+     * Get all fields of a key.
+     *
+     * <p>
+     * Return Code:<ul>
+     * <li>{@link MSException.Code.OK}: success returning the key</li>
+     * <li>{@link MSException.Code.NoKey}: no key found</li>
+     * <li>{@link MSException.Code.IllegalOp}/{@link MSException.Code.ServiceDown}: other issues</li>
+     * </ul></p>
+     *
+     * @param key
+     *          Key Name
+     * @param cb
+     *          Callback to return all fields of the key
+     * @param ctx
+     *          Callback context
+     */
+    public void get(String key, MetastoreCallback<Versioned<Value>> cb, Object ctx);
+
+    /**
+     * Get specified fields of a key.
+     *
+     * <p>
+     * Return Code:<ul>
+     * <li>{@link MSException.Code.OK}: success returning the key</li>
+     * <li>{@link MSException.Code.NoKey}: no key found</li>
+     * <li>{@link MSException.Code.IllegalOp}/{@link MSException.Code.ServiceDown}: other issues</li>
+     * </ul></p>
+     *
+     * @param key
+     *          Key Name
+     * @param fields
+     *          Fields to return
+     * @param cb
+     *          Callback to return specified fields of the key
+     * @param ctx
+     *          Callback context
+     */
+    public void get(String key, Set<String> fields,
+                    MetastoreCallback<Versioned<Value>> cb, Object ctx);
+
+    /**
+     * Update a key according to its version.
+     *
+     * <p>
+     * Return Code:<ul>
+     * <li>{@link MSException.Code.OK}: success updating the key</li>
+     * <li>{@link MSException.Code.BadVersion}: failed to update the key due to bad version</li>
+     * <li>{@link MSException.Code.NoKey}: no key found to update data, if not provided {@link Version.NEW}</li>
+     * <li>{@link MSException.Code.KeyExists}: entry exists providing {@link Version.NEW}</li>
+     * <li>{@link MSException.Code.IllegalOp}/{@link MSException.Code.ServiceDown}: other issues</li>
+     * </ul></p>
+     *
+     * The key is updated only when the version matches its current version.
+     * In particular, if the provided version is:<ul>
+     * <li>{@link Version.ANY}: update the data without comparing its version.
+     *      <b>Note this usage is not encouraged since it may mess up data consistency.</b></li>
+     * <li>{@link Version.NEW}: create the entry if it doesn't exist before;
+     *      Otherwise return {@link MSException.Code.KeyExists}.</li>
+     * </ul>
+     *
+     * @param key
+     *          Key Name
+     * @param value
+     *          Value to update.
+     * @param version
+     *          Version specified to update.
+     * @param cb
+     *          Callback to return new version after updated.
+     * @param ctx
+     *          Callback context
+     */
+    public void put(String key, Value value, Version version, MetastoreCallback<Version> cb, Object ctx);
+
+    /**
+     * Remove a key by its version.
+     *
+     * The key is removed only when the version matches its current version.
+     * If <code>version</code> is {@link Version.ANY}, the key would be removed directly.
+     *
+     * <p>
+     * Return Code:<ul>
+     * <li>{@link MSException.Code.OK}: success updating the key</li>
+     * <li>{@link MSException.Code.NoKey}: if the key doesn't exist.</li>
+     * <li>{@link MSException.Code.BadVersion}: failed to delete the key due to bad version</li>
+     * <li>{@link MSException.Code.IllegalOp}/{@link MSException.Code.ServiceDown}: other issues</li>
+     * </ul></p>
+     *
+     * @param key
+     *          Key Name.
+     * @param version
+     *          Version specified to remove.
+     * @param cb
+     *          Callback to return all fields of the key
+     * @param ctx
+     *          Callback context
+     */
+    public void remove(String key, Version version,
+                       MetastoreCallback<Void> cb, Object ctx);
+
+    /**
+     * Open a cursor to loop over all the entries of the table,
+     * which returns all fields for each entry.
+     * The returned cursor doesn't need to guarantee any order,
+     * since the underlying might be a hash table or an order table.
+     *
+     * @param cb
+     *          Callback to return an opened cursor
+     * @param ctx
+     *          Callback context
+     */
+    public void openCursor(MetastoreCallback<MetastoreCursor> cb, Object ctx);
+
+    /**
+     * Open a cursor to loop over all the entries of the table,
+     * which returns the specified <code>fields</code> for each entry.
+     * The returned cursor doesn't need to guarantee any order,
+     * since the underlying might be a hash table or an order table.
+     *
+     * @param fields
+     *          Fields to select
+     * @param cb
+     *          Callback to return an opened cursor
+     * @param ctx
+     *          Callback context
+     */
+    public void openCursor(Set<String> fields,
+                           MetastoreCallback<MetastoreCursor> cb, Object ctx);
+
+    /**
+     * Close the table.
+     */
+    public void close();
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTableItem.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTableItem.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTableItem.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/MetastoreTableItem.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,70 @@
+/**
+ * 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.metastore;
+
+import org.apache.bookkeeper.versioning.Versioned;
+
+/**
+ * Identify an item in a metastore table.
+ */
+public class MetastoreTableItem {
+
+    private String key;
+    private Versioned<Value> value;
+
+    public MetastoreTableItem(String key, Versioned<Value> value) {
+        this.key = key;
+        this.value = value;
+    }
+
+    /**
+     * Get the key of the table item.
+     *
+     * @return key of table item.
+     */
+    public String getKey() {
+        return key;
+    }
+
+    /**
+     * Set the key of the item.
+     *
+     * @param key Key
+     */
+    public void setKey(String key) {
+        this.key = key;
+    }
+
+    /**
+     * Get the value of the item.
+     *
+     * @return value of the item.
+     */
+    public Versioned<Value> getValue() {
+        return value;
+    }
+
+    /**
+     * Set the value of the item.
+     *
+     * @return value of the item.
+     */
+    public void setValue(Versioned<Value> value) {
+        this.value = value;
+    }
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/Value.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,154 @@
+/**
+ * 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.metastore;
+
+import com.google.common.primitives.UnsignedBytes;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.Collections;
+
+import static org.apache.bookkeeper.metastore.MetastoreTable.ALL_FIELDS;
+
+public class Value {
+    static final Comparator<byte[]> comparator =
+        UnsignedBytes.lexicographicalComparator();
+
+    protected Map<String, byte[]> fields;
+
+    public Value() {
+        fields = new HashMap<String, byte[]>();
+    }
+
+    public Value(Value v) {
+        fields = new HashMap<String, byte[]>(v.fields);
+    }
+
+    public byte[] getField(String field) {
+        return fields.get(field);
+    }
+
+    public Value setField(String field, byte[] data) {
+        fields.put(field, data);
+        return this;
+    }
+
+    public Value clearFields() {
+        fields.clear();
+        return this;
+    }
+
+    public Set<String> getFields() {
+        return fields.keySet();
+    }
+
+    public Map<String, byte[]> getFieldsMap() {
+        return Collections.unmodifiableMap(fields);
+    }
+
+    /**
+     * Select parts of fields.
+     *
+     * @param fields
+     *            Parts of fields
+     * @return new value with specified fields
+     */
+    public Value project(Set<String> fields) {
+        if (ALL_FIELDS == fields) {
+            return new Value(this);
+        }
+        Value v = new Value();
+        for (String f : fields) {
+            byte[] data = this.fields.get(f);
+            v.setField(f, data);
+        }
+        return v;
+    }
+
+    @Override
+    public int hashCode() {
+        HashFunction hf = Hashing.murmur3_32();
+        Hasher hc = hf.newHasher();
+        for (String key : fields.keySet()) {
+            hc.putString(key);
+        }
+        return hc.hash().asInt();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof Value)) {
+            return false;
+        }
+        Value other = (Value) o;
+        if (fields.size() != other.fields.size()) {
+            return false;
+        }
+        for (String f : fields.keySet()) {
+            byte[] v1 = fields.get(f);
+            byte[] v2 = other.fields.get(f);
+            if (0 != comparator.compare(v1, v2)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * Merge other value.
+     *
+     * @param other
+     *          Other Value
+     */
+    public Value merge(Value other) {
+        for (Map.Entry<String, byte[]> entry : other.fields.entrySet()) {
+            if (null == entry.getValue()) {
+                fields.remove(entry.getKey());
+            } else {
+                fields.put(entry.getKey(), entry.getValue());
+            }
+        }
+        return this;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("[");
+        for (Map.Entry<String, byte[]> entry : fields.entrySet()) {
+            String f = entry.getKey();
+            if (null == f) {
+                f = "NULL";
+            }
+            String value;
+            if (null == entry.getValue()) {
+                value = "NONE";
+            } else {
+                value = new String(entry.getValue());
+            }
+            sb.append("('").append(f).append("'=").append(value).append(")");
+        }
+        sb.append("]");
+        return sb.toString();
+    }
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetaStore.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetaStore.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetaStore.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetaStore.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,81 @@
+/**
+ * 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.metastore.mock;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.bookkeeper.metastore.MetaStore;
+import org.apache.bookkeeper.metastore.MetastoreException;
+import org.apache.bookkeeper.metastore.MetastoreTable;
+import org.apache.bookkeeper.metastore.MetastoreScannableTable;
+import org.apache.commons.configuration.Configuration;
+
+public class MockMetaStore implements MetaStore {
+
+    static final int CUR_VERSION = 1;
+
+    static Map<String, MockMetastoreTable> tables =
+        new HashMap<String, MockMetastoreTable>();
+
+    // for test
+    public static void reset() {
+        tables.clear();
+    }
+
+    @Override
+    public String getName() {
+        return getClass().getName();
+    }
+
+    @Override
+    public int getVersion() {
+        return CUR_VERSION;
+    }
+
+    @Override
+    public void init(Configuration conf, int msVersion)
+    throws MetastoreException {
+        // do nothing
+    }
+
+    @Override
+    public void close() {
+        // do nothing
+    }
+
+    @Override
+    public MetastoreTable createTable(String name) {
+        return createMockTable(name);
+    }
+
+    @Override
+    public MetastoreScannableTable createScannableTable(String name) {
+        return createMockTable(name);
+    }
+
+    private MockMetastoreTable createMockTable(String name) {
+        MockMetastoreTable t = tables.get(name);
+        if (t == null) {
+            t = new MockMetastoreTable(this, name);
+            tables.put(name, t);
+        }
+        return t;
+    }
+
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreCursor.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreCursor.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreCursor.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreCursor.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,98 @@
+/**
+ * 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.metastore.mock;
+
+import static org.apache.bookkeeper.metastore.mock.MockMetastoreTable.cloneValue;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.concurrent.ScheduledExecutorService;
+
+import org.apache.bookkeeper.metastore.MSException;
+import org.apache.bookkeeper.metastore.MSException.Code;
+import org.apache.bookkeeper.metastore.MetastoreCursor;
+import org.apache.bookkeeper.metastore.MetastoreTableItem;
+import org.apache.bookkeeper.metastore.Value;
+import org.apache.bookkeeper.versioning.Versioned;
+
+class MockMetastoreCursor implements MetastoreCursor {
+
+    private final ScheduledExecutorService scheduler;
+    private final Iterator<Map.Entry<String, Versioned<Value>>> iter;
+    private final Set<String> fields;
+
+    public MockMetastoreCursor(NavigableMap<String, Versioned<Value>> map, Set<String> fields,
+            ScheduledExecutorService scheduler) {
+        this.iter = map.entrySet().iterator();
+        this.fields = fields;
+        this.scheduler = scheduler;
+    }
+
+    @Override
+    public boolean hasMoreEntries() {
+        return iter.hasNext();
+    }
+
+    @Override
+    public Iterator<MetastoreTableItem> readEntries(int numEntries)
+    throws MSException {
+        if (numEntries < 0) {
+            throw MSException.create(Code.IllegalOp);
+        }
+        return unsafeReadEntries(numEntries);
+    }
+
+    @Override
+    public void asyncReadEntries(final int numEntries, final ReadEntriesCallback cb, final Object ctx) {
+        scheduler.submit(new Runnable() {
+            @Override
+            public void run() {
+                if (numEntries < 0) {
+                    cb.complete(Code.IllegalOp.getCode(), null, ctx);
+                    return;
+                }
+                Iterator<MetastoreTableItem> result = unsafeReadEntries(numEntries);
+                cb.complete(Code.OK.getCode(), result, ctx);
+            }
+        });
+    }
+
+    private Iterator<MetastoreTableItem> unsafeReadEntries(int numEntries) {
+        List<MetastoreTableItem> entries = new ArrayList<MetastoreTableItem>();
+        int nCount = 0;
+        while (iter.hasNext() && nCount < numEntries) {
+            Map.Entry<String, Versioned<Value>> entry = iter.next();
+            Versioned<Value> value = entry.getValue();
+            Versioned<Value> vv = cloneValue(value.getValue(), value.getVersion(), fields);
+            String key = entry.getKey();
+            entries.add(new MetastoreTableItem(key, vv));
+            ++nCount;
+        }
+        return entries.iterator();
+    }
+
+    @Override
+    public void close() throws IOException {
+        // do nothing
+    }
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreTable.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreTable.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreTable.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/metastore/mock/MockMetastoreTable.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,341 @@
+/**
+ * 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.metastore.mock;
+
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import org.apache.bookkeeper.metastore.MSException.Code;
+import org.apache.bookkeeper.metastore.MetastoreCallback;
+import org.apache.bookkeeper.metastore.MetastoreCursor;
+import org.apache.bookkeeper.metastore.MetastoreScannableTable;
+import org.apache.bookkeeper.metastore.Value;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+
+public class MockMetastoreTable implements MetastoreScannableTable {
+
+    public static class MockVersion implements Version {
+        int version;
+
+        public MockVersion(int v) {
+            this.version = v;
+        }
+
+        public MockVersion(MockVersion v) {
+            this.version = v.version;
+        }
+
+        public synchronized MockVersion incrementVersion() {
+            ++version;
+            return this;
+        }
+
+        @Override
+        public Occurred compare(Version v) {
+            if (null == v) {
+                throw new NullPointerException("Version is not allowed to be null.");
+            }
+            if (v == Version.NEW) {
+                return Occurred.AFTER;
+            } else if (v == Version.ANY) {
+                return Occurred.CONCURRENTLY;
+            } else if (!(v instanceof MockVersion)) {
+                throw new IllegalArgumentException("Invalid version type");
+            }
+            MockVersion mv = (MockVersion)v;
+            int res = version - mv.version;
+            if (res == 0) {
+                return Occurred.CONCURRENTLY;
+            } else if (res < 0) {
+                return Occurred.BEFORE;
+            } else {
+                return Occurred.AFTER;
+            }
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (null == obj ||
+                !(obj instanceof MockVersion)) {
+                return false;
+            }
+            MockVersion v = (MockVersion)obj;
+            return 0 == (version - v.version);
+        }
+
+        @Override
+        public String toString() {
+            return "version=" + version;
+        }
+
+        @Override
+        public int hashCode() {
+            return version;
+        }
+    }
+
+    private String name;
+    private TreeMap<String, Versioned<Value>> map = null;
+    private ScheduledExecutorService scheduler;
+
+    public MockMetastoreTable(MockMetaStore metastore, String name) {
+        this.map = new TreeMap<String, Versioned<Value>>();
+        this.name = name;
+        this.scheduler = Executors.newSingleThreadScheduledExecutor();
+    }
+
+    @Override
+    public String getName () {
+        return this.name;
+    }
+
+    static Versioned<Value> cloneValue(Value value, Version version, Set<String> fields) {
+        if (null != value) {
+            Value newValue = new Value();
+            if (ALL_FIELDS == fields) {
+                fields = value.getFields();
+            }
+            for (String f : fields) {
+                newValue.setField(f, value.getField(f));
+            }
+            value = newValue;
+        }
+
+        if (null == version) {
+            throw new NullPointerException("Version isn't allowed to be null.");
+        }
+        if (Version.ANY != version && Version.NEW != version) {
+            if (version instanceof MockVersion) {
+                version = new MockVersion(((MockVersion)version).version);
+            } else {
+                throw new IllegalStateException("Wrong version type.");
+            }
+        }
+        return new Versioned<Value>(value, version);
+    }
+
+    @Override
+    public void get(final String key, final MetastoreCallback<Versioned<Value>> cb, final Object ctx) {
+        scheduler.submit(new Runnable() {
+            @Override
+            public void run() {
+                scheduleGet(key, ALL_FIELDS, cb, ctx);
+            }
+        });
+    }
+
+    @Override
+    public void get(final String key, final Set<String> fields, final MetastoreCallback<Versioned<Value>> cb,
+            final Object ctx) {
+        scheduler.submit(new Runnable() {
+            @Override
+            public void run() {
+                scheduleGet(key, fields, cb, ctx);
+            }
+        });
+    }
+
+    public synchronized void scheduleGet(String key, Set<String> fields, MetastoreCallback<Versioned<Value>> cb,
+            Object ctx) {
+        if (null == key) {
+            cb.complete(Code.IllegalOp.getCode(), null, ctx);
+            return;
+        }
+        Versioned<Value> vv = mockGet(key);
+        int rc = null == vv ? Code.NoKey.getCode() : Code.OK.getCode();
+        if (vv != null) {
+            vv = cloneValue(vv.getValue(), vv.getVersion(), fields);
+        }
+        cb.complete(rc, vv, ctx);
+    }
+
+    @Override
+    public void put(final String key, final Value value, final Version version, final MetastoreCallback<Version> cb,
+            final Object ctx) {
+        scheduler.submit(new Runnable() {
+            @Override
+            public void run() {
+                if (null == key || null == value || null == version) {
+                    cb.complete(Code.IllegalOp.getCode(), null, ctx);
+                    return;
+                }
+                Result<Version> result = mockPut(key, value, version);
+                cb.complete(result.code.getCode(), result.value, ctx);
+            }
+        });
+    }
+
+    @Override
+    public void remove(final String key, final Version version, final MetastoreCallback<Void> cb, final Object ctx) {
+        scheduler.submit(new Runnable() {
+            @Override
+            public void run() {
+                if (null == key || null == version) {
+                    cb.complete(Code.IllegalOp.getCode(), null, ctx);
+                    return;
+                }
+                Code code = mockRemove(key, version);
+                cb.complete(code.getCode(), null, ctx);
+            }
+        });
+    }
+
+    @Override
+    public void openCursor(MetastoreCallback<MetastoreCursor> cb, Object ctx) {
+        openCursor(EMPTY_START_KEY, true, EMPTY_END_KEY, true, Order.ASC,
+                   ALL_FIELDS, cb, ctx);
+    }
+
+    @Override
+    public void openCursor(Set<String> fields,
+                           MetastoreCallback<MetastoreCursor> cb, Object ctx) {
+        openCursor(EMPTY_START_KEY, true, EMPTY_END_KEY, true, Order.ASC,
+                   fields, cb, ctx);
+    }
+
+    @Override
+    public void openCursor(String firstKey, boolean firstInclusive,
+                           String lastKey, boolean lastInclusive,
+                           Order order, MetastoreCallback<MetastoreCursor> cb,
+                           Object ctx) {
+        openCursor(firstKey, firstInclusive, lastKey, lastInclusive,
+                   order, ALL_FIELDS, cb, ctx);
+    }
+
+    @Override
+    public void openCursor(final String firstKey, final boolean firstInclusive,
+                           final String lastKey, final boolean lastInclusive,
+                           final Order order, final Set<String> fields,
+                           final MetastoreCallback<MetastoreCursor> cb, final Object ctx) {
+        scheduler.submit(new Runnable() {
+            @Override
+            public void run() {
+                Result<MetastoreCursor> result = mockOpenCursor(firstKey, firstInclusive, lastKey, lastInclusive,
+                        order, fields);
+                cb.complete(result.code.getCode(), result.value, ctx);
+            }
+        });
+    }
+
+    private synchronized Versioned<Value> mockGet(String key) {
+        return map.get(key);
+    }
+
+    private synchronized Code mockRemove(String key, Version version) {
+        Versioned<Value> vv = map.get(key);
+        if (null == vv) {
+            return Code.NoKey;
+        }
+        if (Version.Occurred.CONCURRENTLY != vv.getVersion().compare(version)) {
+            return Code.BadVersion;
+        }
+        map.remove(key);
+        return Code.OK;
+    }
+
+    static class Result<T> {
+        Code code;
+        T value;
+
+        public Result(Code code, T value) {
+            this.code = code;
+            this.value = value;
+        }
+    }
+
+    private synchronized Result<Version> mockPut(String key, Value value, Version version) {
+        Versioned<Value> vv = map.get(key);
+        if (vv == null) {
+            if (Version.NEW != version) {
+                return new Result<Version>(Code.NoKey, null);
+            }
+            vv = cloneValue(value, version, ALL_FIELDS);
+            vv.setVersion(new MockVersion(0));
+            map.put(key, vv);
+            return new Result<Version>(Code.OK, new MockVersion(0));
+        }
+        if (Version.NEW == version) {
+            return new Result<Version>(Code.KeyExists, null);
+        }
+        if (Version.Occurred.CONCURRENTLY != vv.getVersion().compare(version)) {
+            return new Result<Version>(Code.BadVersion, null);
+        }
+        vv.setVersion(((MockVersion)vv.getVersion()).incrementVersion());
+        vv.setValue(vv.getValue().merge(value));
+        return new Result<Version>(Code.OK, new MockVersion((MockVersion)vv.getVersion()));
+    }
+
+    private synchronized Result<MetastoreCursor> mockOpenCursor(
+            String firstKey, boolean firstInclusive,
+            String lastKey, boolean lastInclusive,
+            Order order, Set<String> fields) {
+        if (0 == map.size()) {
+            return new Result<MetastoreCursor>(Code.OK, MetastoreCursor.EMPTY_CURSOR);
+        }
+
+        boolean isLegalCursor = false;
+        NavigableMap<String, Versioned<Value>> myMap = null;
+        if (Order.ASC == order) {
+            myMap = map;
+            if (EMPTY_END_KEY == lastKey ||
+                lastKey.compareTo(myMap.lastKey()) > 0) {
+                lastKey = myMap.lastKey();
+                lastInclusive = true;
+            }
+            if (EMPTY_START_KEY == firstKey ||
+                firstKey.compareTo(myMap.firstKey()) < 0) {
+                firstKey = myMap.firstKey();
+                firstInclusive = true;
+            }
+            if (firstKey.compareTo(lastKey) <= 0) {
+                isLegalCursor = true;
+            }
+        } else if (Order.DESC == order) {
+            myMap = map.descendingMap();
+            if (EMPTY_START_KEY == lastKey ||
+                lastKey.compareTo(myMap.lastKey()) < 0) {
+                lastKey = myMap.lastKey();
+                lastInclusive = true;
+            }
+            if (EMPTY_END_KEY == firstKey ||
+                firstKey.compareTo(myMap.firstKey()) > 0) {
+                firstKey = myMap.firstKey();
+                firstInclusive = true;
+            }
+            if (firstKey.compareTo(lastKey) >= 0) {
+                isLegalCursor = true;
+            }
+        }
+
+        if (!isLegalCursor || null == myMap) {
+            return new Result<MetastoreCursor>(Code.IllegalOp, null);
+        }
+        MetastoreCursor cursor = new MockMetastoreCursor(
+                myMap.subMap(firstKey, firstInclusive, lastKey, lastInclusive), fields, scheduler);
+        return new Result<MetastoreCursor>(Code.OK, cursor);
+    }
+
+    @Override
+    public void close() {
+        // do nothing
+    }
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreScannableTableAsyncToSyncConverter.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreScannableTableAsyncToSyncConverter.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreScannableTableAsyncToSyncConverter.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreScannableTableAsyncToSyncConverter.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,59 @@
+/**
+ * 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.metastore;
+
+import java.util.Set;
+
+import org.apache.bookkeeper.metastore.MetastoreScannableTable.Order;
+
+public class MetastoreScannableTableAsyncToSyncConverter extends
+             MetastoreTableAsyncToSyncConverter {
+
+    private MetastoreScannableTable scannableTable;
+
+    public MetastoreScannableTableAsyncToSyncConverter(
+            MetastoreScannableTable table) {
+        super(table);
+        this.scannableTable = table;
+    }
+
+    public MetastoreCursor openCursor(String firstKey, boolean firstInclusive,
+                                      String lastKey, boolean lastInclusive,
+                                      Order order)
+    throws MSException {
+        HeldValue<MetastoreCursor> retValue = new HeldValue<MetastoreCursor>();
+        // make the actual async call
+        this.scannableTable.openCursor(firstKey, firstInclusive, lastKey, lastInclusive,
+                                       order, retValue, null);
+        retValue.waitCallback();
+        return retValue.getValue();
+    }
+
+    public MetastoreCursor openCursor(String firstKey, boolean firstInclusive,
+                                      String lastKey, boolean lastInclusive,
+                                      Order order, Set<String> fields)
+    throws MSException {
+        HeldValue<MetastoreCursor> retValue = new HeldValue<MetastoreCursor>();
+        // make the actual async call
+        this.scannableTable.openCursor(firstKey, firstInclusive, lastKey, lastInclusive,
+                                       order, fields, retValue, null);
+        retValue.waitCallback();
+        return retValue.getValue();
+    }
+
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreTableAsyncToSyncConverter.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreTableAsyncToSyncConverter.java?rev=1407520&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreTableAsyncToSyncConverter.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/metastore/MetastoreTableAsyncToSyncConverter.java Fri Nov  9 16:13:10 2012
@@ -0,0 +1,131 @@
+/**
+ * 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.metastore;
+
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.bookkeeper.metastore.MetastoreCallback;
+import org.apache.bookkeeper.metastore.MetastoreTable;
+import org.apache.bookkeeper.metastore.MSException;
+import org.apache.bookkeeper.metastore.MSException.Code;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+
+// Converts async calls to sync calls for MetastoreTable. Currently not
+// intended to be used other than for simple functional tests, however,
+// could be developed into a sync API.
+
+public class MetastoreTableAsyncToSyncConverter {
+
+    static class HeldValue<T> implements MetastoreCallback<T> {
+        private CountDownLatch countDownLatch = new CountDownLatch(1);
+        private int code;
+        private T value = null;
+
+        void waitCallback() throws MSException {
+            try {
+                countDownLatch.await(10, TimeUnit.SECONDS);
+            } catch (InterruptedException ie) {
+                throw MSException.create(Code.InterruptedException);
+            }
+
+            if (Code.OK.getCode() != code) {
+                throw MSException.create(Code.get(code));
+            }
+        }
+
+        public T getValue() {
+            return value;
+        }
+
+        @Override
+        public void complete(int rc, T value, Object ctx) {
+            this.code = rc;
+            this.value = value;
+            countDownLatch.countDown();
+        }
+    }
+
+    protected MetastoreTable table;
+
+    public MetastoreTableAsyncToSyncConverter(MetastoreTable table) {
+        this.table = table;
+    }
+
+    public Versioned<Value> get(String key) throws MSException {
+        HeldValue<Versioned<Value>> retValue =
+            new HeldValue<Versioned<Value>>();
+
+        // make the actual async call
+        this.table.get(key, retValue, null);
+
+        retValue.waitCallback();
+        return retValue.getValue();
+    }
+
+    public Versioned<Value> get(String key, Set<String> fields)
+    throws MSException {
+        HeldValue<Versioned<Value>> retValue =
+            new HeldValue<Versioned<Value>>();
+
+        // make the actual async call
+        this.table.get(key, fields, retValue, null);
+
+        retValue.waitCallback();
+        return retValue.getValue();
+    }
+
+    public void remove(String key, Version version) throws MSException {
+        HeldValue<Void> retValue = new HeldValue<Void>();
+
+        // make the actual async call
+        this.table.remove(key, version, retValue, null);
+
+        retValue.waitCallback();
+    }
+
+    public Version put(String key, Value value, Version version)
+    throws MSException {
+        HeldValue<Version> retValue = new HeldValue<Version>();
+
+        // make the actual async call
+        this.table.put(key, value, version, retValue, null);
+
+        retValue.waitCallback();
+        return retValue.getValue();
+    }
+
+    public MetastoreCursor openCursor() throws MSException {
+        HeldValue<MetastoreCursor> retValue = new HeldValue<MetastoreCursor>();
+        // make the actual async call
+        this.table.openCursor(retValue, null);
+        retValue.waitCallback();
+        return retValue.getValue();
+    }
+
+    public MetastoreCursor openCursor(Set<String> fields) throws MSException {
+        HeldValue<MetastoreCursor> retValue = new HeldValue<MetastoreCursor>();
+        // make the actual async call
+        this.table.openCursor(fields, retValue, null);
+        retValue.waitCallback();
+        return retValue.getValue();
+    }
+
+}