You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@bookkeeper.apache.org by Enrico Olivelli <eo...@gmail.com> on 2017/10/06 11:54:18 UTC

Re: [apache/bookkeeper] Issue-605 BP-15 New CreateLedger API (#510)

the "conversation" page on GitHub now is broken....too many comments!!!
https://github.com/apache/bookkeeper/pull/510

Answerns inline in this email
I can create a new PR eventually

2017-10-05 21:56 GMT+02:00 Sijie Guo <no...@github.com>:

> *@sijie* commented on this pull request.
> ------------------------------
>
> In bookkeeper-common/src/main/java/org/apache/bookkeeper/
> common/concurrent/FutureUtils.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143017588>:
>
> > @@ -15,7 +15,6 @@
>   * See the License for the specific language governing permissions and
>   * limitations under the License.
>   */
> -
>
> The changes in file doesn't see to be necessary, right?
>

reverted


> ------------------------------
>
> In bookkeeper-server/pom.xml
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143018035>:
>
> > @@ -188,6 +188,41 @@
>        </extension>
>      </extensions>
>      <plugins>
> +    <!--
>
> if this isn't needed in this change, let's remove it. we can add it later.
>

I am going to uncomment these lines, it is the only way to enable
checkstyle on the requested packages (api + impl)



> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/
> BookKeeperException.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143018480>:
>
> > + * 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.client.api;
> +
> +import org.apache.bookkeeper.client.LedgerHandleAdv;
> +
> +/**
> + * Super class for all errors which occur using BookKeeper client
> + *
> + * @since 4.6
> + */
> +public abstract class BookKeeperException extends Exception {
>
> I think I have a comment before to keep the name as BKException.
>

done


> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/BookKeeper.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143020668>:
>
> > @@ -816,7 +866,7 @@ public LedgerHandle createLedger(int ensSize, int writeQuorumSize, int ackQuorum
>       * @param ackQuorumSize
>       * @param digestType
>       * @param passwd
> -     * @param customMetadata
>
> any ideas on removing customMetadata?
>

it is not a parameter of that method, maybe it was a cut and paste


> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/LedgerEntry.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143024999>:
>
> > @@ -80,7 +86,7 @@ public long getLength() {
>       *
>       * @return an InputStream which gives access to the content of the entry
>       * @throws IllegalStateException if this method is called twice
> -     */
> +     */
>
> remove spaces
>
done

> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/LedgerHandle.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143025168>:
>
> > @@ -312,17 +318,24 @@ void writeLedgerConfig(GenericCallback<Void> writeCb) {
>
>      /**
>       * Close this ledger synchronously.
> +     *
>
> inheritdoc ?
>
done

> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/LedgerHandle.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143025513>:
>
> >       * @see #asyncClose
>       */
> +    @Override
>      public void close()
>
> do you need this? Handle already have a default implementation of #close
> using #asyncClose
>
yes, I tried to drop it, but this actaully will break source compatibilty a
lot, because legacy close uses legacy BKException, if I drop this method it
wlil thow "new" BKException.
We can save this.
I have no strong opinion, I can drop if you prefer


> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/LedgerHandle.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143026489>:
>
> > @@ -628,6 +686,16 @@ public long addEntry(byte[] data) throws InterruptedException, BKException {
>          return addEntry(data, 0, data.length);
>      }
>
> +    @Override
> +    public CompletableFuture<Long> append(ByteBuf data) {
>
> can you just call #asyncAddEntry(data, callback, ctx)? logic such as
> retain bytebuf and construct pending add op is already handled by
> #asyncAddEntry. you don't need to duplicate this logic.
>
done


> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/LedgerHandle.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143028801>:
>
> > +    @Override
> +    public CompletableFuture<Long> tryReadLastAddConfirmed() {
> +        CompletableFuture<Long> result = new CompletableFuture<>();
> +
> +        ReadLastConfirmedCallback callback = (int rc, long lastConfirmed, Object ctx) -> {
> +            SyncCallbackUtils.finish(rc, lastConfirmed, result);
> +        };
> +        asyncTryReadLastConfirmed(callback, result);
> +        return result;
> +    }
> +
> +    @Override
> +    public CompletableFuture<Long> readLastAddConfirmed() {
> +        CompletableFuture<Long> result = new CompletableFuture<>();
> +
> +        ReadLastConfirmedCallback callback = (int rc, long lastConfirmed, Object ctx) -> {
>
> ReadLastConfirmedCallback in line 1041 and line 1053 seem to be same. Can
> we follow the same pattern as how we handle other callbacks?
>
good catch
I have added a new FutureReadLastConfirmed in SyncCallbackUtils.
For some "callback + future" combo I have used 'extends' in order to
preserve a memory allocation (like for Add and ReadLastConfirmed) which are
operations which are done very often
I can use the same pattern everywhere if you prefer

> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/LedgerHandleAdv.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143031590>:
>
> > @@ -225,6 +226,21 @@ public void safeRun() {
>          }
>      }
>
> +    @Override
> +    public CompletableFuture<Long> write(long entryId, ByteBuf data) {
>
> can we try to add one? I like to make sure all these method share same
> code path, rather than duplicating. otherwise it is going to be hard to
> maintain.
>
done, I have added a new private method with ByteBuf and I am calling the
new method from the byte[] API, so we have only one write path

> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/LedgerMetadata.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143031683>:
>
> > @@ -193,7 +194,8 @@ boolean hasPassword() {
>          return hasPassword;
>      }
>
> -    byte[] getPassword() {
> +    @VisibleForTesting
> +    public byte[] getPassword() {
>
> fine for me.
> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/SyncCallbackUtils.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143032451>:
>
> > +     * @param <T>
> +     * @param rc
> +     * @param result
> +     * @param future
> +     */
> +    public static <T> void finish(int rc, T result, CompletableFuture<T> future) {
> +        if (rc != BKException.Code.OK) {
> +            future.completeExceptionally(BKException.create(rc).fillInStackTrace());
> +        } else {
> +            future.complete(result);
> +        }
> +    }
> +
> +    static class SyncCreateCallback implements AsyncCallback.CreateCallback {
> +
> +        private final CompletableFuture future;
>
> I think I suggested adding type, no?
>
using "super" does not work.
if you use <T super LedgerHandle> it won't work for the new API, if you use
<T super WriteHandle> it won't work for WriteAdvHandle...and so on.
If you want this I have to create at least 3 different callbacks


> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/SyncCallbackUtils.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143032708>:
>
> > +        /**
> +         * Create callback implementation for synchronous create call.
> +         *
> +         * @param rc return code
> +         * @param lh ledger handle object
> +         * @param ctx optional control object
> +         */
> +        @Override
> +        @SuppressWarnings(value = "unchecked")
> +        public void createComplete(int rc, LedgerHandle lh, Object ctx) {
> +            SyncCallbackUtils.finish(rc, lh, future);
> +        }
> +
> +    }
> +
> +    static class SyncOpenCallback<T> implements AsyncCallback.OpenCallback {
>
> you don't need <T>
>
done

> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/SyncCallbackUtils.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143032939>:
>
> > +         *
> +         * @param rc return code
> +         * @param lh ledger handle object
> +         * @param ctx optional control object
> +         */
> +        @Override
> +        @SuppressWarnings(value = "unchecked")
> +        public void createComplete(int rc, LedgerHandle lh, Object ctx) {
> +            SyncCallbackUtils.finish(rc, lh, future);
> +        }
> +
> +    }
> +
> +    static class SyncOpenCallback<T> implements AsyncCallback.OpenCallback {
> +
> +        private final CompletableFuture future;
>
> doesn't CompletableFuture<? super LedgerHandle> work? I am just curious --
> I was suggesting adding type to ensure the right completable future is
> passed in.
>

explained above


> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/SyncCallbackUtils.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143033041>:
>
> > +         *          return code
> +         * @param lh
> +         *          ledger handle
> +         * @param ctx
> +         *          optional control object
> +         */
> +        @Override
> +        @SuppressWarnings("unchecked")
> +        public void openComplete(int rc, LedgerHandle lh, Object ctx) {
> +            SyncCallbackUtils.finish(rc, lh, future);
> +        }
> +    }
> +
> +    static class SyncDeleteCallback implements AsyncCallback.DeleteCallback {
> +
> +        private final CompletableFuture future;
>
> same comment as above.
>

done


> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/SyncCallbackUtils.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143033798>:
>
> > +         */
> +        @Override
> +        public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) {
> +            LedgerHandle.LastConfirmedCtx lcCtx = (LedgerHandle.LastConfirmedCtx) ctx;
> +
> +            synchronized(lcCtx) {
> +                lcCtx.setRC(rc);
> +                lcCtx.setLastConfirmed(lastConfirmed);
> +                lcCtx.notify();
> +            }
> +        }
> +    }
> +
> +    static class SyncCloseCallback implements AsyncCallback.CloseCallback {
> +
> +        private final CompletableFuture future;
>
> same comment as above.
>

done

------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/api/DigestType.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143034062>:
>
> > + *
> + *   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.client.api;
> +
> +/**
> + * Digest type.
> + */
>
> @since 4.6
>

done

> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/
> BookKeeperBuilderImpl.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143034534>:
>
> > +    private final org.apache.bookkeeper.client.BookKeeper.Builder builder;
> +
> +    public BookKeeperBuilderImpl(ClientConfiguration conf) {
> +        this.builder = org.apache.bookkeeper.client.BookKeeper.forConfig(conf);
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder eventLoopGroup(EventLoopGroup component) {
> +        Preconditions.checkNotNull(component);
> +        builder.eventLoopGroup(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder zk(ZooKeeper component) {
> +        Preconditions.checkNotNull(component);
>
> it is okay to set a null zk, right?
>

done, here an for comments below

> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/
> BookKeeperBuilderImpl.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143034853>:
>
> > +/**
> + * Internal builder for {@link org.apache.bookkeeper.client.api.BookKeeper} client.
> + *
> + * @since 4.6
> + */
> +public class BookKeeperBuilderImpl implements BookKeeperBuilder {
> +
> +    private final org.apache.bookkeeper.client.BookKeeper.Builder builder;
> +
> +    public BookKeeperBuilderImpl(ClientConfiguration conf) {
> +        this.builder = org.apache.bookkeeper.client.BookKeeper.forConfig(conf);
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder eventLoopGroup(EventLoopGroup component) {
> +        Preconditions.checkNotNull(component);
>
> it is okay to set a null eventloop.
>
> and I don't think you need to validate the parameters on each method. in
> builder pattern, people usually validate the parameters on #build()
> function.
> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/
> BookKeeperBuilderImpl.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143034901>:
>
> > +
> +/**
> + * Internal builder for {@link org.apache.bookkeeper.client.api.BookKeeper} client.
> + *
> + * @since 4.6
> + */
> +public class BookKeeperBuilderImpl implements BookKeeperBuilder {
> +
> +    private final org.apache.bookkeeper.client.BookKeeper.Builder builder;
> +
> +    public BookKeeperBuilderImpl(ClientConfiguration conf) {
> +        this.builder = org.apache.bookkeeper.client.BookKeeper.forConfig(conf);
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder eventLoopGroup(EventLoopGroup component) {
>
> rename component?
> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/
> BookKeeperBuilderImpl.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143035094>:
>
> > +    public BookKeeperBuilder eventLoopGroup(EventLoopGroup component) {
> +        Preconditions.checkNotNull(component);
> +        builder.eventLoopGroup(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder zk(ZooKeeper component) {
> +        Preconditions.checkNotNull(component);
> +        builder.zk(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder statsLogger(StatsLogger component) {
> +        Preconditions.checkNotNull(component);
>
> validate stats logger on #build()
>

this is already done in legacy builder, I will drop this kind of lines


> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/
> BookKeeperBuilderImpl.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143035245>:
>
> > +    public BookKeeperBuilder zk(ZooKeeper component) {
> +        Preconditions.checkNotNull(component);
> +        builder.zk(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder statsLogger(StatsLogger component) {
> +        Preconditions.checkNotNull(component);
> +        builder.statsLogger(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder dnsResolver(DNSToSwitchMapping component) {
> +        Preconditions.checkNotNull(component);
>
> I believe it is okay to pass a null dns resolver as well.
> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/
> BookKeeperBuilderImpl.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143035314>:
>
> > +    public BookKeeperBuilder statsLogger(StatsLogger component) {
> +        Preconditions.checkNotNull(component);
> +        builder.statsLogger(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder dnsResolver(DNSToSwitchMapping component) {
> +        Preconditions.checkNotNull(component);
> +        builder.dnsResolver(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder requestTimer(HashedWheelTimer component) {
> +        Preconditions.checkNotNull(component);
>
> it is okay to be null as well.
> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/client/impl/
> BookKeeperBuilderImpl.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143035435>:
>
> > +    public BookKeeperBuilder dnsResolver(DNSToSwitchMapping component) {
> +        Preconditions.checkNotNull(component);
> +        builder.dnsResolver(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder requestTimer(HashedWheelTimer component) {
> +        Preconditions.checkNotNull(component);
> +        builder.requestTimer(component);
> +        return this;
> +    }
> +
> +    @Override
> +    public BookKeeperBuilder featureProvider(FeatureProvider component) {
> +        Preconditions.checkNotNull(component);
>
> I think it is okay to be null as well.
> ------------------------------
>
> In bookkeeper-server/src/main/java/org/apache/bookkeeper/
> client/impl/package-info.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143035631>:
>
> > + * 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.
> + *
> + */
> +/**
> + * BookKeeper Client implementation package
>
> add '.' end of this sentence.
>
> also did you enable checkstyle for api and impl?
>

done, but in order to enable checkstyle I have added checkstyle to pom.xml


> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/
> client/MockBookKeeperTestCase.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143036128>:
>
> > +                @Override
> +                @SuppressWarnings("unchecked")
> +                public ArrayList<BookieSocketAddress> answer(InvocationOnMock invocation) throws Throwable {
> +                    Object[] args = invocation.getArguments();
> +                    int ensembleSize = (Integer) args[0];
> +                    return generateNewEnsemble(ensembleSize);
> +                }
> +            });
> +    }
> +
> +    protected void setupBookieClientAddEntry() {
> +        doAnswer((Answer) (InvocationOnMock invokation) -> {
> +            Object[] args = invokation.getArguments();
> +            BookkeeperInternalCallbacks.WriteCallback callback = (BookkeeperInternalCallbacks.WriteCallback) args[5];
> +            BookieSocketAddress bookieSocketAddress = (BookieSocketAddress) args[0];
> +            long _ledgerId = (Long) args[1];
>
> I don't think we use "_xyz" for variables. I would stick to be just
> "ledgerId" to be consistent. I see this pattern has been used in multiple
> places in this class.
>

done, sorry

------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/
> client/MockBookKeeperTestCase.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143036223>:
>
> > +                }
> +            });
> +    }
> +
> +    protected void setupBookieClientAddEntry() {
> +        doAnswer((Answer) (InvocationOnMock invokation) -> {
> +            Object[] args = invokation.getArguments();
> +            BookkeeperInternalCallbacks.WriteCallback callback = (BookkeeperInternalCallbacks.WriteCallback) args[5];
> +            BookieSocketAddress bookieSocketAddress = (BookieSocketAddress) args[0];
> +            long _ledgerId = (Long) args[1];
> +            long _entryId = (Long) args[3];
> +            Object ctx = args[6];
> +
> +            submit(() -> {
> +                boolean fenced = fencedLedgers.contains(_ledgerId);
> +                LOG.error("addEntry {}@{} fenced {}", _ledgerId, _entryId, fenced);
>
> why this is error?
>

dropped, it was my debug


> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/
> client/api/BookKeeperApiTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143038188>:
>
> > +    public void testWriteHandle() throws Exception {
> +        Map<String, byte[]> customMetadata = new HashMap<>();
> +        customMetadata.put("test", "test".getBytes(StandardCharsets.UTF_8));
> +        try (WriteHandle writer
> +            = result(
> +                newCreateLedgerOp()
> +                    .withAckQuorumSize(1)
> +                    .withWriteQuorumSize(2)
> +                    .withEnsembleSize(3)
> +                    .withDigestType(DigestType.MAC)
> +                    .withCustomMetadata(customMetadata)
> +                    .withPassword("password".getBytes(StandardCharsets.UTF_8))
> +                    .execute());) {
> +
> +            byte[] data = "foo".getBytes(StandardCharsets.UTF_8);
> +            writer.append(ByteBuffer.wrap(data)).get();
>
>
>    1. what do you want to test between line 65 to line 69? why not write
>    it in a loop? what is the difference between line 67 and other lines?
>
>
refactored the suite, sometimes I needed to call write/append methods to
advance LAC


>
>    1. you might consider verifying the mock BookieClient on how many
>    times it is called?
>
>
Don't know if this will introduce flakyness, I can try if you prefer


>
>
> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/
> client/api/BookKeeperApiTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143038360>:
>
> > +        Map<String, byte[]> customMetadata = new HashMap<>();
> +        customMetadata.put("test", "test".getBytes(StandardCharsets.UTF_8));
> +        try (WriteAdvHandle writer
> +            = result(newCreateLedgerOp()
> +                .withAckQuorumSize(1)
> +                .withWriteQuorumSize(2)
> +                .withEnsembleSize(3)
> +                .withDigestType(DigestType.MAC)
> +                .withCustomMetadata(customMetadata)
> +                .withPassword("password".getBytes(StandardCharsets.UTF_8))
> +                .makeAdv()
> +                .execute());) {
> +
> +            long entryId = 0;
> +            byte[] data = "foo".getBytes(StandardCharsets.UTF_8);
> +            writer.write(entryId++, ByteBuffer.wrap(data)).get();
>
> I am not sure what does line 92 to line 96 do. why they are different?
>

fixed

> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/
> client/api/BookKeeperApiTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143038709>:
>
> > +        try (WriteAdvHandle writer
> +            = result(newCreateLedgerOp()
> +                .withAckQuorumSize(1)
> +                .withWriteQuorumSize(2)
> +                .withEnsembleSize(3)
> +                .withDigestType(DigestType.MAC)
> +                .withCustomMetadata(customMetadata)
> +                .withPassword("password".getBytes(StandardCharsets.UTF_8))
> +                .makeAdv()
> +                .withLedgerId(1234)
> +                .execute());) {
> +
> +            assertEquals(1234, writer.getId());
> +
> +            long entryId = 0;
> +            byte[] data = "foo".getBytes(StandardCharsets.UTF_8);
>
> I don't see the difference between testWriteAdvHandle and
> testWriteAdvHandleWithFixedLedgerId. I image you need to verify the
> different behavior between without and with ledger id. but the test cases
> don't actually do this validation.
>

fixed

> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/
> client/api/BookKeeperApiTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143039328>:
>
> > +                assertEquals(1234, writer.getId());
> +
> +                long entryId = 0;
> +                byte[] data = "foo".getBytes(StandardCharsets.UTF_8);
> +                writer.write(entryId++, ByteBuffer.wrap(data)).get();
> +                writer.write(entryId++, ByteBuffer.wrap(data)).get();
> +                writer.write(entryId++, Unpooled.wrappedBuffer(data)).get();
> +                writer.write(entryId++, ByteBuffer.wrap(data)).get();
> +                long expectedEntryId = writer.write(entryId++, ByteBuffer.wrap(data)).get();
> +                assertEquals(expectedEntryId, writer.getLastAddConfirmed());
> +            }
> +        }
> +    }
> +
> +    @Test
> +    public void testOpenLedger() throws Exception {
>
>
>    1.
>
>    can you just break this test into multiple test cases?
>    testOpenLedgerUnauthorized(), testOpenLedgerDigestUnmatched,
>    testOpenLedgerNoSuchLedger, ... It is much clear than putting everything
>    into one single test.
>
>
done


>
>    1.
>    2.
>
>    you don't need to write data. line 189 to line 192 provide the entries
>    for read.
>
>
in some test I needed to perform writes to see the LAC advancing


>
>    1.
>
> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/
> client/api/BookKeeperApiTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143039578>:
>
> > +            assertEquals(2, reader.readLastAddConfirmed().get().intValue());
> +            assertEquals(2, reader.tryReadLastAddConfirmed().get().intValue());
> +
> +            checkEntries(reader.read(0, reader.getLastAddConfirmed()).get(), data);
> +            checkEntries(reader.readUnconfirmed(0, reader.getLastAddConfirmed()).get(), data);
> +        }
> +    }
> +
> +    @Test
> +    public void testOpenLedgerWithRecovery() throws Exception {
> +        long lId;
> +        byte[] data = "foo".getBytes(StandardCharsets.UTF_8);
> +        final byte[] password = "password".getBytes(StandardCharsets.UTF_8);
> +        Map<String, byte[]> customMetadata = new HashMap<>();
> +        customMetadata.put("test", "test".getBytes(StandardCharsets.UTF_8));
> +        try (WriteHandle writer = result(newCreateLedgerOp()
>
> why do you need writehandle in this test case? registerMockEntryForRead
> already provides entries for reading, no?
>

in some test I needed to perform writes to see the LAC advancing an to make
recovery really work


> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/
> client/api/BookKeeperApiTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143039721>:
>
> > +                try (ReadHandle reader = result(bkc.newOpenLedgerOp()
> +                        .withDigestType(DigestType.MAC)
> +                        .withPassword("password".getBytes(StandardCharsets.UTF_8))
> +                        .withRecovery(false)
> +                        .withLedgerId(lId)
> +                        .execute())) {
> +                    assertEquals(1, reader.getLastAddConfirmed());
> +                    assertEquals(1, reader.readLastAddConfirmed().get().intValue());
> +                    checkEntries(reader.read(0, reader.getLastAddConfirmed()).get(), data);
> +                }
> +            }
> +        }
> +    }
> +
> +    @Test
> +    public void testDeleteLedger() throws Exception {
>
> can you break this into smaller test cases?
>

done

> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/
> BookKeeperBuildersTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143040312>:
>
> > +        WriteHandle writer = newCreateLedgerOp()
> +            .withAckQuorumSize(ackQuorumSize)
> +            .withEnsembleSize(ensembleSize)
> +            .withWriteQuorumSize(writeQuorumSize)
> +            .withCustomMetadata(customMetadata)
> +            .withPassword(password)
> +            .execute()
> +            .get();
> +        assertEquals(ledgerId, writer.getId());
> +        LedgerMetadata metadata = getLedgerMetadata(ledgerId);
> +        assertEquals(ensembleSize, metadata.getEnsembleSize());
> +        assertEquals(ackQuorumSize, metadata.getAckQuorumSize());
> +        assertEquals(writeQuorumSize, metadata.getWriteQuorumSize());
> +        assertArrayEquals(password, metadata.getPassword());
> +
> +        try {
>
> break this into multiple tests cases? and use @Test(expected =
> BKIncorrectParameterException.class). It is much clearer on what you are
> testing.
> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/
> BookKeeperBuildersTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143040589>:
>
> > +            fail("shoud not be able to create a ledger with such specs");
> +        } catch (BKIncorrectParameterException err) {
> +        }
> +
> +        closeBookkeeper();
> +        try {
> +            result(newCreateLedgerOp()
> +                .withPassword(password)
> +                .execute());
> +            fail("shoud not be able to create a ledger, client is closed");
> +        } catch (BKClientClosedException err) {
> +        }
> +    }
> +
> +    @Test
> +    public void testCreateAdvLedger() throws Exception {
>
> same comment as above.
>
> and ensembleSize, writeQuorumSize, ackQuorumSize, ledgerId, password,
> customMetadata are used across different test cases. you can define them as
> class fields.
>

done

------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/
> BookKeeperBuildersTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143040649>:
>
> > +            .execute()).getId());
> +
> +        closeBookkeeper();
> +        try {
> +            result(newCreateLedgerOp()
> +                .withPassword(password)
> +                .makeAdv()
> +                .execute());
> +            fail("shoud not be able to create a ledger, client is closed");
> +        } catch (BKClientClosedException err) {
> +        }
> +
> +    }
> +
> +    @Test
> +    public void testOpenLedger() throws Exception {
>
> same comment as above
> ------------------------------
>
> In bookkeeper-server/src/test/java/org/apache/bookkeeper/client/api/
> BookKeeperBuildersTest.java
> <https://github.com/apache/bookkeeper/pull/510#discussion_r143040703>:
>
> > +            .withLedgerId(ledgerId)
> +            .withRecovery(false)
> +            .execute());
> +        closeBookkeeper();
> +        try {
> +            result(newOpenLedgerOp()
> +                .withLedgerId(ledgerId)
> +                .execute());
> +            fail("shoud not be able to open a ledger, client is closed");
> +        } catch (BKClientClosedException err) {
> +        }
> +
> +    }
> +
> +    @Test
> +    public void testDeleteLedger() throws Exception {
>
> same comment as above.
>
> —
> You are receiving this because you were mentioned.
> Reply to this email directly, view it on GitHub
> <https://github.com/apache/bookkeeper/pull/510#pullrequestreview-67467664>,
> or mute the thread
> <https://github.com/notifications/unsubscribe-auth/AJB8tgmHJ7LIUmpzhrw8GwZ_wnq7gp2Uks5spTR7gaJpZM4PXY6Y>
> .
>