/** * Add entry synchronously to an open ledger. * * @param data array of bytes to be written * @return the id of the newly appended entry */ default long append(ByteBuffer data) throws BKException, InterruptedException { return append(Unpooled.wrappedBuffer(data)); }
/** * Add an entry asynchronously to an open ledger. * * @param data array of bytes to be written * @param offset the offset in the bytes array * @param length the length of the bytes to be appended * @return a completable future represents the add result, in case of success the future returns the entry id * of this newly appended entry */ default CompletableFuture<Long> appendAsync(byte[] data, int offset, int length) { return appendAsync(Unpooled.wrappedBuffer(data, offset, length)); }
@Override protected void run(BookKeeper bk, Flags flags) throws Exception { byte[] data = new byte[100]; // test data try (WriteHandle wh = result(bk.newCreateLedgerOp() .withEnsembleSize(flags.ensembleSize) .withWriteQuorumSize(flags.writeQuorumSize) .withAckQuorumSize(flags.ackQuorumSize) .withDigestType(DigestType.CRC32C) .withPassword(new byte[0]) .execute())) { System.out.println("Ledger ID: " + wh.getId()); long lastReport = System.nanoTime(); for (int i = 0; i < flags.numEntries; i++) { wh.append(data); if (TimeUnit.SECONDS.convert(System.nanoTime() - lastReport, TimeUnit.NANOSECONDS) > 1) { System.out.println(i + " entries written"); lastReport = System.nanoTime(); } } System.out.println(flags.numEntries + " entries written to ledger " + wh.getId()); } } }
/** * Synchronous close the write handle, any adds in flight will return errors. * * <p>Closing a ledger will ensure that all clients agree on what the last * entry of the ledger is. Once the ledger has been closed, all reads from the * ledger will return the same set of entries. * * <p>The close operation can error if it finds conflicting metadata when it * tries to write to the metadata store. On close, the metadata state is set to * closed and lastEntry and length of the ledger are fixed in the metadata. A * conflict occurs if the metadata in the metadata store has a different value for * the lastEntry or length. If another process has updated the metadata, setting it * to closed, but have fixed the lastEntry and length to the same values as this * process is trying to write, the operation completes successfully. */ @Override default void close() throws BKException, InterruptedException { FutureUtils.<Void, BKException>result(closeAsync(), BKException.HANDLER); } }
/** * Add an entry asynchronously to an open ledger. * * @param data array of bytes to be written * @return a completable future represents the add result, in case of success the future returns the entry id * of this newly appended entry */ default CompletableFuture<Long> appendAsync(byte[] data) { return appendAsync(Unpooled.wrappedBuffer(data)); }
/** * Add an entry synchronously to an open ledger. * * @param data array of bytes to be written * @param offset the offset in the bytes array * @param length the length of the bytes to be appended * @return the entry id of this newly appended entry */ default long append(byte[] data, int offset, int length) throws BKException, InterruptedException { return append(Unpooled.wrappedBuffer(data, offset, length)); }
/** * Add entry asynchronously to an open ledger. * * @param data array of bytes to be written * @return an handle to the result, in case of success it will return the id of the newly appended entry */ default CompletableFuture<Long> appendAsync(ByteBuffer data) { return appendAsync(Unpooled.wrappedBuffer(data)); }
/** * Add an entry synchronously to an open ledger. * * @param data array of bytes to be written * @return the entry id of this newly appended entry */ default long append(byte[] data) throws BKException, InterruptedException { return append(Unpooled.wrappedBuffer(data)); }
/** * Add entry synchronously to an open ledger. * * @param data a bytebuf to be written. The bytebuf's reference count will be decremented by 1 after the * call completes. * @return the id of the newly appended entry */ default long append(ByteBuf data) throws BKException, InterruptedException { return FutureUtils.<Long, BKException>result(appendAsync(data), BKException.HANDLER); }