Re: Improve the process of removing bookies from a cluster

2021-09-09 Thread Ivan Kelly
> Do you think it will be different enough from the
> autorecovery process to put it on the bookie being drained or should it
> still reside within the autorecovery process?
I think there's enough commonality that a single solution can be
applied to both.
At root, both have to copy entries and update metadata. The questions
are only really about where this happens.

-Ivan


Re: Improve the process of removing bookies from a cluster

2021-09-08 Thread Ivan Kelly
Hi Yang,

> Besides the auditor, I think the external operator (whether a human
> operator or an automation program) also cares about the "draining" state of
> a bookie.

This isn't a question of the internal model, but of how it is exposed.
API-wise, it would not be a problem to expose draining as an endpoint
as a bookie http endpoint, but ultimately that should call out to an
endpoint in the auditor. "draining" isn't so much a state but an
operation, that should only take place in the read-only state, and be
performed by the auditor. The auditor sees and records that there is a
draining operation active on the bookie. It should oversee that data
is copied off of the bookie.

> If the data is expected to be moved off the bookie by auto-recovery, the
> bookie has to be set as "draining" to kick off the data migration, and
> there should also be APIs to mark a bookie as "draining" and to check if
> the bookie is in the "draining" state. Although "draining" is a special
> case of "readonly", would it be more clear to make it another possible
> value of `BookieMode` and provide similar APIs as for the readonly state?

No, I think bookie mode should be limited to read_write and read_only.
read-only is of interest to both the client and the bookie. The client
needs to know which bookies are read-only so that it does not select
it for writes. The bookie needs to know that it is read-only so that
it doesn't accept new writes. The bookie doesn't care if there's a
draining operation happening. It will just see read traffic.

> Or do you have any suggestions on the management of the draining state and
> relative APIs?

I would add endpoint /api/v1/bookie/drain API.
POST to this API calls out to the auditor to create a drain operation
for that API and returns an ID.
GET /api/v1/bookie/drain/ returns the status of the drain
operation (calling out to the auditor).
DELETE /api/v1/bookie/drain/ cancels the drain operation.

One open question is how to decide that a drain is done? The existing
autorecovery code does some really horrible things using zookeeper as
a queue, and then modifies the ledger metadata at the end to remove
the offending bookie. Using zookeeper as a queue is bad, and
replication workers end up racing for locks. But this can decide when
a bookie is empty because the bookie no longer exists in the metadata
for any ledger. Thinking about it more, even if you did use the
current underreplication stuff, it would need to be modified to allow
it to copy from a live bookie, because I think it doesn't allow data
from live bookies to be rereplicated.
I think a better solution is to use the same mechanism as data
integrity outlined earlier in the thread.
Let's say you have a ledger L1, and you want to decommission bookie B4
(there are 3 other bookies).
Metadata looks like {ensembles: [{0: B1, B2, B4}]}
When you want to decommission B4, the auditor updates the metadata to
{ensembles: [{0: B1, B2, B4}], replacements: [{ensemble: 0, from: B4,
to:B3}]}
Data integrity kicks in on B3 (it can even be prompted to start by the
auditor). It sees that it should have the data from ensemble 0, so
does the copy, preferring B4 for the reads.
Once the copy is complete, B3 updates the metadata to {ensembles: [{0:
B1, B2, B3}]}.
So decommissioning costs 2 ZK writes per ledger.
To cancel the decommission, the replacements field is deleted from the metadata.

-Ivan


Re: Improve the process of removing bookies from a cluster

2021-09-08 Thread Ivan Kelly
> I am not very familiar with bookkeeper and auditor history (so please let
> me know if this understanding doesn't work), but it seems to me that the
> process responsible for draining the bookie could be local to the bookie
> itself to limit network hops.

This is a very good point. One of the reasons we did the data
integrity work was that if there were entries missing from a bookie,
they would have to be copied to a replicator process and then copied
to the destination. The data integrity checker (which I promise we
will push upstream soon), runs on the bookie and only does one copy.
>From another node to the local node.

One thing that running the draining on the local bookie doesn't cover,
is that, if the bookie is down and unrecoverable, the bookie will
never be drained, so the data on the bookie would remain
underreplicated.

Perhaps this is a different case, and needs to be handled differently,
but it could also be handled by a mechanism similar to the data
integrity. There could be an auditor like process that scans all
ledger metadata to find ledger segments where any of the bookies are
missing ("draining" could be considered another missing state). When
it finds a ledger with unreplicated data, it selects a bookie from the
remaining bookies to take the data. This bookie is then marked as an
extra replica in the metadata. From this point the mechanism is the
data integrity mechanism. A bookie periodically checks if it has all
the data it is supposed to have (taking the extra replica metadata
into account), and copies anything that is missing.

> Do you see the bookkeeper tiered storage being used in every case?
No, I doubt it. For us, the thinking is that we want to use tiered
storage anyhow. So if it's there, we may as well use it for
autoscaling, and not spend too many cycles on another mechanism.

Another aspect of this is cost. Without tiered storage, the variable
that decides the number of nodes you need is the
(throughput)*(retention). So, assuming that throughput doesn't change
much, or is cyclical, there'll be very few autoscaling decisions taken
(nodes need to stick around for retention).
If you use tiered storage, the number of nodes needed is purely based
on throughput. You'll have fewer bookies, but autoscaling will need to
respond more frequently to variations in throughput.

-Ivan


Re: Improve the process of removing bookies from a cluster

2021-09-07 Thread Ivan Kelly
Hi Yang,

> Autoscaling is exactly one motivation for me to bring this topic up. I
> understand that the auto-recovery is not perfect at the moment, but it's an
> important component that maintains the core invariants of a bookkeeper
> cluster, so I think we may keep improving it until we find a better
> replacement.

Internally we have replaced auto recovery with another mechanism that
checks that the bookie
has all the data it says it has. We have plans to push this upstream
in the next month or two. A side
effect of the change is that it allows you to run without journal safely.
However, it doesn't cover the decommission usecase. For decommission,
our thinking is that once we
have tiered storage at the bookie level, the decommission story
becomes a lot easier. Basically, you
switch to read-only and wait for tiered storage to clear it out, even
bumping the bookies ledgers in priority
for offloading to tiered storage. We're still early in this process
(utilization metrics have to come first).

> I'm thinking maybe we can put the "draining" state as a special member in
> the properties of `BookieServiceInfo
> `,
> and let the auditor check the properties of readonly bookies to see if a
> bookie need to be drained and seen as unavailable.

"draining" state is not something that anyone but the auditor needs to
care about. It's not a state attribute of the bookie,
but instead an external service's opinion of what should be happening
with the bookie. as such, it doesn't belong in the bookie service info
itself.

It should go in some metadata that the auditor is maintaining about
the bookie, so that when the auditor restarts, it can see that it
should prioritize
moving data from that bookie.

> The bookie state API
> 
> might
> also be enhanced to support updating and persisting the state of a bookie
> dynamically. And a new API might need to be added to check if all ledgers
> have been moved off a "draining" bookie. Do you think these changes make
> sense?

This API is a bit of a hodge podge of different kinds of state.
"shutting down", "running" and "availableForHighPriority" are
transient states. We need read-only to be persistent if it is
requested by an external entity. To persist the readonly state, there
are multiple options. We can persist it to zookeeper somehow, or we
can persist it as file on the bookie's disk. But the API linked isn't
really suited in any case.
What we need is a /bookie/state/readonly endpoint, where we PUT and
payload like '{"readonly": true}'. When this gets a OK response, that
state should be persisted, so any reboot will keep the bookie in the
correct state.

-Ivan


Re: Decommission command without stopping bookkeeper server

2021-09-07 Thread Ivan Kelly
This seems to be part of another thread "Improve the process of
removing bookies from a cluster". Can you please respond there to keep
the conversation in one place?

-Ivan


On Tue, Sep 7, 2021 at 12:57 PM zhangao  wrote:
>
> Hello everyone,Now, decommission command need bookie stoped, and only 
> support one bookie at same time. I plan to make decommission behave more like 
> HDFS decommission commands:   1、support decommission two or more servers at 
> same time.   2、decommission will make servers become read-only. (more robust: 
> only support replication read.)   3、I also have implemented replication 
> throttle, so replication will not include too much load on clusters. see
> 3、when decommission finished, the server is ready to stop.


Re: Improve the process of removing bookies from a cluster

2021-09-06 Thread Ivan Kelly
Hi Yang,

This is something we've been thinking about internally. It's
especially important if we want to implement auto scaling for bookies.

I'm not sure we need a "draining" state as such. Or at least, the
draining state doesn't need to be at the same level as "read-only".
"draining" is only interesting to the entity moving data off of the
bookie, so the auditor could keep a record that it is draining bookie
X,
but the cluster as a whole doesn't need to care about it.

>From a logical point of view, decommissioning/scale down should be a matter of
1. Mark bookie as read only so that no new data is added to it
2. Wait for the bookie to hold no live data

The most important thing, and a thing that we have really missed in
the past is the ability to mark a running bookie as read-only.
This should be trivial to implement, though the split between
bookie-shell and bk-cli is a bit of a mess right now. I think there is
a
REST api endpoint, but it is non-persistent.

Once the bookie is read-only, we have the following options for
getting live data off of the bookie.
1. Wait for pulsar retention period to pass (currently available, but
can take a long time).
2. Use tiered storage to move older data off. This is currently
implemented as a pulsar feature, but I think it would make sense to
move it down to the bookie layer.
3. Use auditor/auto recovery to move the data.

Personally I'm not a fan of the auditor/auto recovery stuff currently
in bookkeeper. Any time we've relied on it in the past it has
blown up on us or moved too slowly to be very useful. Part of the
problem is that it conflates data integrity checking, with bookie
decommissioning. Data integrity is concerned with ensuring a bookie
has the data zookeeper says it has.
Decommissioning is moving data off of a bookie. One should be
naturally cheap, the other expensive. With autorecovery, they both
end up expensive.

A problem with both tiered storage and autorecovery for
decommissioning, is that they need to move data and so induce load in
the
system. However, this load isn't well quantified, so they use manually
set rate limiting, which doesn't respond to the rest of the load
in the system. The first thing  we need to do, and we are actively
working on this, is to generate accurate utilization and saturation
metrics for bookies. Once we have these metrics, the entity copying
the data can do so much faster without impacting production
traffic.

Cheers,
Ivan


Re: Skip writes to the journal part 2 - BP-44

2021-08-19 Thread Ivan Kelly
Hi Enrico,

I meant to reply yesterday but forgot. There's still one change we
need to make before pushing this code up. The change is fairly small,
so it shouldn't take too long.

Cheers,
Ivan

On Wed, Aug 18, 2021 at 1:13 PM Enrico Olivelli  wrote:
>
> Ivan and Jack,
> we have recently committed the change to bypass writes to the journal.
>
> In the scope of that work there was the idea that we needed client side
> changes and also other bookie side changes to deal with the possibility
> that a Bookie loses its data.
>
> this is the BP for reference
> https://github.com/apache/bookkeeper/pull/2706
>
> What is your plan for contributing to the second part of that work ?
>
> Only disabling the journal is not safe and it is very dangerous
>
> The discussion on the PR is very long and I am not sure we reached a
> general consensus.
>
> If you want we can organize a Virtual Community Meeting to talk face to face
>
> Thanks for contributing this important feature
> Enrico


Re: Docker images and security vunerabilities

2021-08-12 Thread Ivan Kelly
How did it end up on centos in the first place?

+1 for moving to ubuntu.

-Ivan

On Thu, Aug 12, 2021 at 1:03 PM Enrico Olivelli  wrote:
>
> Hello folks,
> I have found a PR [1] that is about upgrading the base image to Centos 8
> because the Centos 7 image has some reported vulnerabilities.
>
> I believe that upgrading to Centos 8 could be fine, but it is not enough to
> cover this kind of problem.
>
> In the Apache Pulsar community we recently switched to Ubuntu [2] and we
> also added an automatic update of the system dependencies while building
> the image with (apt-get -y dist-upgrade).
>
> I propose to switch the base docker image to Ubuntu following Pulsar PR [2].
>
> The only downside will be for users that are using the BK docker image as
> base image, as Ubuntu is very different from Centos and so consumers of the
> image will have to do some work while upgrading.
>
> Thoughts ?
>
> Enrico
>
> [1] https://github.com/apache/bookkeeper/pull/2756
> [2] https://github.com/apache/pulsar/pull/11026


Re: Contributing Splunk changes back to OSS

2020-11-11 Thread Ivan Kelly
> There might be slight overlap with BP-40 which I have in the works for
> Audit logging for BKShell and Bkctl.

Where is BP-40? I don't see it on the dev list.

-Ivan


Re: Contributing Splunk changes back to OSS

2020-11-11 Thread Ivan Kelly
> Based on feedback on this, I'd also like to later start a similar
> Gradle proposal for Pulsar builds too.
Yes, but let's wait and see how it goes here first.

I'll start putting together a BP today.

-Ivan


Re: Contributing Splunk changes back to OSS

2020-11-10 Thread Ivan Kelly
> > What are peoples opinions on moving BookKeeper to gradle (assuming
> > I/splunk do the legwork)?
> > If people are open to it, I'll submit a BP.
> >
>
> +1. My only question is how do you do an Apache release. I'd like to see BP
> covering that question.

Yes, this will need a BP to cover all the CI + release things.
This is a good thing though. It'll give us a chance to clear house.

-Ivan


Contributing Splunk changes back to OSS

2020-11-10 Thread Ivan Kelly
Hi folks,

It's been about a year since Streamlio joined Splunk and since then
we've had a bit of forking with our BK branch.
It has gotten to a stage where it's starting to be a problem for us,
so we'd like to start to get things back in sync.

There are a couple of big chunks of work to come back.
We've added a data integrity checker that replaces a lot of the
functionality of autorecovery and allows us to run without a journal.
We refactored the bookie to allow dependency injection.
We've rewritten the entry logger to use direct I/O (allowing 2GBps
writes per bookie).

One other thing we've done is to change the build system to use gradle.
The major driver for this was that maven is just slow, even before you
start running tests.
"mvn clean package -DskipTests" takes 4m30 on my laptop. "./gradlew
clean jar" takes 40s.
Subsequent builds on gradle are much much faster, as it does
incremental building.
Incremental building exists in maven, but it doesn't work.
Gradle also handle multimodule projects better. If I make a change in
bookkeeper-common,
"./gradlew :bookkeeper-server:test" will pick up the change without
having to explicitly
"mvn install" the bookkeeper-common. In my opinion it's just a much
nicer build system
to work with. Even the poms it generates are better as they avoid
dependency pollution.

What are peoples opinions on moving BookKeeper to gradle (assuming
I/splunk do the legwork)?
If people are open to it, I'll submit a BP.

Another thing that BK (and the whole ecosystem) is missing is
structured logging.
We also plan to add structured logging to BK in soon. This is a major
motivator for converging the branches,
as it touches a lot of places.

Anyhow, any feedback appreciated.

-Ivan


Re: Bypassing writes to the Journal - everybody wants this feature !

2020-11-10 Thread Ivan Kelly
> The limit of the given patches is that it is simply skipping all of the
> writes to the journal,  and this in turn is a big problem:
> - if you restart the bookie it is likely that you lose your data, and
> especially the 'fenced' flag
> - clients cannot rely on most of the guarantees that BK provides

There are two problems (restatement of above).
- A bookie may accept writes for a ledger which it has previously
promised not to (loss of the fenced bit)
- A bookie may reply negatively for the read of a ledgers entry, which
is has previously acknowledged receipt of (breaks consistency
guarantees)

In both cases, the problem is unclosed ledgers. If the bookie, when it
starts, can detect a non-clean shutdown.
If it does, it can find all unclosed ledgers which were writing to it, and
a) accept not more writes
b) not reply negatively to requests for entries of those ledgers which
do not exist on the bookie.

a) is similar in effect to fencing. If a client was actively writing
to the ledger, it should have updated the ensemble by that time in any
case.
b) is a new concept (lets call it limbo). If the entries do not exist
locally, they may still have existed previously. So to respond
negatively would be untrue
and messes up the recovery process.

As you mentioned, splunk already has this change internally. I'm going
to start another thread about that.

In summary, skipping the journal is fine if you have some other things
in place. However, I would make it a cluster wide property.
If we say skipping the journal is safe (due to multi AZ and the extra
checks) then it should be safe for all.


-Ivan

















 Also another problem is that those implementations work on a per-bookie
> basis, I understand that the user in those cases is Pulsar and usually you
> do not share your BK cluster with other applications (is it really true ?
> think about PulsarFunctions and BK StreamStorage service).
>
> Btw this is not true for our case at EmailSuccess.com and also at
> MagNews.com, in which we are sharing the bookies with other components
> (like HerdDB, DistributedLog, BlobIt).
>
> Skipping the journal is a good trade off in several cases, because it makes
> writes blazing fast and also reduces the write amplification.
>
> I would like to wrap up all of this stuff and provide a feature to BK, to
> be used consistently by all of the users.
>
> I think that it will be far better to have a WriteFlag to enable this
> feature, this way different clients will be able to express their
> durability constraints and service level regarding this feature.
>
> Also when the Bookie is not writing to the Journal, after a restart, we
> should tell to the clients that the Bookie is not able to return data for a
> given ledger or to tell if the ledger has been fenced. IIUC Ivan and Matteo
> already have this change in their private fork.
>
>
> I will be happy to start a BP or to help any other volunteer in writing it.
> We should work as a community on this topic.
>
> Thoughts ?
> Enrico


Pushed branch to apache/bookkeeper by mistake

2020-03-04 Thread Ivan Kelly
Hi folks,

I pushed a branch to the wrong repo by mistake. I meant to push to my
fork. Deleted now. Please ignore, and sorry for the noise.

-Ivan


Re: Log truncation and sync up when bookie fails and rejoins

2020-01-28 Thread Ivan Kelly
> Thanks for the detailed response. Just one question, if writer doesn't
> fail, but bookie write fails (Say a soft failure because of network problem
> or GC pause), the writer will create a new fragment within a ledger. So the
> same sequence of operations that happen while closing the ledger needs to
> happen at fragment level as well. Because the log entry can be copied to
> new fragment, the log from failed bookie (or more of soft-failed bookie
> because of network issue or GC pause) in previous fragment needs to be
> truncated.

In the case of a bookie failing during write, there's no need for the
recovery part of the process. The writer already knows which entry was
the last entry acknowledged before the failure. The write to update
the ensemble, i.e. create the new fragment, is similar as the write to
close the ledger. it fixes the last entry of the previous segment.

In terms of truncating, closing a fragment is not the same as closing
a ledger. Take the following scenario.

Writer w1
Bookie b1, b2

b1: e1, 
b2: e1, e2

w1 is writing to a ledger, and b1 crashes before e2 is acknowledged.
w1 detects the crash and changes the ensemble to b3,b2, and then
promptly crashes. Recovery at this point may then recover the end of
the ledger as e2, even though the end of the fragment was e1. This is
fine, but it demonstrates how in a ensemble change truncation does not
occur.

-Ivan


Re: Log truncation and sync up when bookie fails and rejoins

2020-01-28 Thread Ivan Kelly
> So.. log truncation, the way it's needed in leader based systems like RAFT
> and Kafka, where leader may have entries appended to its log which are not
> replicated. If leader crashes before replicating entries, which will elect
> other node as leader. Once the previous leader rejoins the cluster, it
> needs to truncate its own log removing all the conflicting entries. This
> case wont happen in bookkeeper?

Something similar does happen in bookkeeper. Firstly, it's important
to keep in mind that a single ledger in bookkeeper only has a single
writer ever. If the writer crashes, no new entries can be added to
that ledger. In this way, you can kinda think of a ledger as a term in
RAFT or an epoch in ZK. To build a replicated log in bookkeeper, you
must chain a bunch of ledgers together. BK leaves that to the user.

In the case of a writer crash, the next writer(i.e. the client adding
the next ledger to the chain) needs to run the recovery algorithm,
which finds the last entry which may possibly have been acknowledged
to the reader. It uses this last entry to mark the ledger as closed.
This "close" operation is similar to a truncate. Individual bookies in
the ensemble may have entries past this last entry. However, these
entries do not exist on enough bookies for the entry to have been
acknowledged as written, so they can be ignored.

For example, say you have a ledger A across 3 bookies, b1 and b2, and
being written to by writer w1, with ensemble 2, write quorum 2 and ack
quorum 2.

w1 crashes when the bookies have the following entries.

b1: e1
b2: e1, e2

The next writer, w2, could close this ledger at either e1 or e2. Both
are correct.
For e1, it would try to read the last entry from both b1 & b2, but
only b1 would reply. w2 would see that e1 is the last entry on b1 and
as ack quorum is 2, it no entry beyond e1 has been acknowledged to w1
(to acknowledge to the writer, acknowledgement must be received from
|ack quorum| bookies).
For e2, it would try to read the last entry from both b1 & b2, either
b2 or both would reply. If both replied w2 would see that e2 was
written by the client, but not acknowledged to w1. However, it is also
possible that only b2 replied, so w2 cannot divine whether e2 was
acknowledged to w1. In both cases, it's safe to take e2 as the last
entry. w2 ensures that e2 is replicated to |ack quorum| bookies, and
marks it as the end of the ledger.

The case where e1 was found to be the last ledger can be considered
similar to truncate.

-Ivan


Re: Log truncation and sync up when bookie fails and rejoins

2020-01-28 Thread Ivan Kelly
> From the bookie perspective, if a bookie of a ledger ensemble crashes while a 
> ledger is being written to, then it is replaced and the history of the ledger 
> is updated in the ledger metadata according to the last add confirmed by the 
> crashed bookie. If the bookie crashes after the ledger is closed, then 
> auto-recovery re-replicates the data.

There is one exception to this for the case where the ledger is being
actively written to. When the ack quorum is smaller than the write
quorum. In this case, when a bookie fails, entries between the point
of failure and the point of failure detection will only be replicated
|ackQuorum| times, not |writeQuorum| times. Autorecovery should
eventually pick this up.

-Ivan


Re: Log truncation and sync up when bookie fails and rejoins

2020-01-28 Thread Ivan Kelly
> But who takes care of updating a particular Bookie in case it crashses (or
> temporarily partitioned) and rejoins the cluster?

Autorecovery takes care of this. The metadata describes the entries
that should exist on a bookie. If this doesn't match what actually
exists on the bookie, autorecovery rereplicates it.

-Ivan


Re: Still Failing: apache/bookkeeper#4573 (master - f89e3fb)

2019-06-03 Thread Ivan Kelly
Actually, I can't merge master into it because it's sijie's branch.
Sijie will have to do so.

On Mon, Jun 3, 2019 at 1:51 PM Ivan Kelly  wrote:
>
> Master has been failing for a long time. A failing PR was merged in
> https://github.com/apache/bookkeeper/pull/2066
>
> The fix is available in
> https://github.com/apache/bookkeeper/pull/2102. I'll merge master into
> that. It should pass and we can merge.
>
> -Ivan
>
> On Fri, May 31, 2019 at 7:29 PM Enrico Olivelli  wrote:
> >
> > It seems that travis is not very happy
> >
> > Enrico
> >
> > -- Forwarded message -
> > Da: Travis CI 
> > Date: ven 31 mag 2019, 18:48
> > Subject: Still Failing: apache/bookkeeper#4573 (master - f89e3fb)
> > To: 
> >
> >
> > apache
> >
> > /
> >
> > bookkeeper
> > <https://travis-ci.org/apache/bookkeeper?utm_medium=notification_source=email>
> >
> > [image: branch icon]master
> > <https://github.com/apache/bookkeeper/tree/master>
> > [image: build has failed]
> > Build #4573 is still failing
> > <https://travis-ci.org/apache/bookkeeper/builds/539745986?utm_medium=notification_source=email>
> > [image: arrow to build time]
> > [image: clock icon]22 mins and 53 secs
> >
> > [image: karanmehta93 avatar]karanmehta93
> > f89e3fb CHANGESET →
> > <https://github.com/apache/bookkeeper/compare/26e8004e6b93...f89e3fbb751f>
> >
> > Update lastLogMark to EOF when replaying journal
> >
> > Descriptions of the changes in this PR:
> >
> > ### Motivation
> >
> > The [commit](
> > https://github.com/apache/bookkeeper/commit/36be8362399341022c8de64f9319270726df2cb3)
> > caused integration test failure `test101_RegenerateIndex`, with the
> > exception
> > ```
> > ```java.io.IOException: Invalid argument
> > at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
> > at sun.nio.ch.FileDispatcherImpl.read(FileDispatcherImpl.java:46)
> > at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> > at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> > at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:159)
> > at
> > org.apache.bookkeeper.bookie.JournalChannel.read(JournalChannel.java:257)
> > at org.apache.bookkeeper.bookie.Journal.fullRead(Journal.java:1171)
> > at org.apache.bookkeeper.bookie.Journal.scanJournal(Journal.java:792)
> > at org.apache.bookkeeper.bookie.Bookie.replay(Bookie.java:924)
> > at org.apache.bookkeeper.bookie.Bookie.readJournal(Bookie.java:886)
> > at org.apache.bookkeeper.bookie.Bookie.start(Bookie.java:943)
> > at org.apache.bookkeeper.proto.BookieServer.start(BookieServer.java:141)
> > at
> > org.apache.bookkeeper.server.service.BookieService.doStart(BookieService.java:58)
> >
> > at
> > org.apache.bookkeeper.common.component.AbstractLifecycleComponent.start(AbstractLifecycleComponent.java:78)
> >
> > at
> > org.apache.bookkeeper.common.component.LifecycleComponentStack.lambda$start$2(LifecycleComponentStack.java:113)
> >
> > at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:408)
> > at
> > org.apache.bookkeeper.common.component.LifecycleComponentStack.start(LifecycleComponentStack.java:113)
> >
> > at
> > org.apache.bookkeeper.common.component.ComponentStarter.startComponent(ComponentStarter.java:80)
> >
> > at org.apache.bookkeeper.server.Main.doMain(Main.java:229)
> > at org.apache.bookkeeper.server.Main.main(Main.java:203)
> > ```
> >
> > As discussed on slack, it is hard to figure out an exact reason as to why
> > the native JNI call fails with an invalid argument. Hence this PR proposes
> > that the `lastLogMark` is updated to journal EOF instead of an arbitrary
> > LONG.MAX_VALUE. The FileChannel interface defines that the implementors can
> > pass in any long offset and the file handler should return EOF immediately
> > when trying to read it. However it doesn't seem to be working as expected.
> >
> > ### Changes
> >
> > Updated `Journal#setLastLogMark()` method to accept an `scanOffset` instead
> > of constant `LONG.MAX_VALUE`.
> >
> > ivankelly eolivelli
> >
> > Reviewers: Ivan Kelly , Enrico Olivelli <
> > eolive...@gmail.com>
> >
> > This closes #2105 from karanmehta93/master
> >
> > Want to know about upcoming build environment updates?
> >
> > Would you like to stay up-to-date with the upcoming Travis CI build
> > environment updates? We set up a mailing list for you!
> > SIGN UP HERE <http://eepurl.com/9OCsP>
>

Re: Still Failing: apache/bookkeeper#4573 (master - f89e3fb)

2019-06-03 Thread Ivan Kelly
Master has been failing for a long time. A failing PR was merged in
https://github.com/apache/bookkeeper/pull/2066

The fix is available in
https://github.com/apache/bookkeeper/pull/2102. I'll merge master into
that. It should pass and we can merge.

-Ivan

On Fri, May 31, 2019 at 7:29 PM Enrico Olivelli  wrote:
>
> It seems that travis is not very happy
>
> Enrico
>
> -- Forwarded message -
> Da: Travis CI 
> Date: ven 31 mag 2019, 18:48
> Subject: Still Failing: apache/bookkeeper#4573 (master - f89e3fb)
> To: 
>
>
> apache
>
> /
>
> bookkeeper
> <https://travis-ci.org/apache/bookkeeper?utm_medium=notification_source=email>
>
> [image: branch icon]master
> <https://github.com/apache/bookkeeper/tree/master>
> [image: build has failed]
> Build #4573 is still failing
> <https://travis-ci.org/apache/bookkeeper/builds/539745986?utm_medium=notification_source=email>
> [image: arrow to build time]
> [image: clock icon]22 mins and 53 secs
>
> [image: karanmehta93 avatar]karanmehta93
> f89e3fb CHANGESET →
> <https://github.com/apache/bookkeeper/compare/26e8004e6b93...f89e3fbb751f>
>
> Update lastLogMark to EOF when replaying journal
>
> Descriptions of the changes in this PR:
>
> ### Motivation
>
> The [commit](
> https://github.com/apache/bookkeeper/commit/36be8362399341022c8de64f9319270726df2cb3)
> caused integration test failure `test101_RegenerateIndex`, with the
> exception
> ```
> ```java.io.IOException: Invalid argument
> at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
> at sun.nio.ch.FileDispatcherImpl.read(FileDispatcherImpl.java:46)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.FileChannelImpl.read(FileChannelImpl.java:159)
> at
> org.apache.bookkeeper.bookie.JournalChannel.read(JournalChannel.java:257)
> at org.apache.bookkeeper.bookie.Journal.fullRead(Journal.java:1171)
> at org.apache.bookkeeper.bookie.Journal.scanJournal(Journal.java:792)
> at org.apache.bookkeeper.bookie.Bookie.replay(Bookie.java:924)
> at org.apache.bookkeeper.bookie.Bookie.readJournal(Bookie.java:886)
> at org.apache.bookkeeper.bookie.Bookie.start(Bookie.java:943)
> at org.apache.bookkeeper.proto.BookieServer.start(BookieServer.java:141)
> at
> org.apache.bookkeeper.server.service.BookieService.doStart(BookieService.java:58)
>
> at
> org.apache.bookkeeper.common.component.AbstractLifecycleComponent.start(AbstractLifecycleComponent.java:78)
>
> at
> org.apache.bookkeeper.common.component.LifecycleComponentStack.lambda$start$2(LifecycleComponentStack.java:113)
>
> at com.google.common.collect.ImmutableList.forEach(ImmutableList.java:408)
> at
> org.apache.bookkeeper.common.component.LifecycleComponentStack.start(LifecycleComponentStack.java:113)
>
> at
> org.apache.bookkeeper.common.component.ComponentStarter.startComponent(ComponentStarter.java:80)
>
> at org.apache.bookkeeper.server.Main.doMain(Main.java:229)
> at org.apache.bookkeeper.server.Main.main(Main.java:203)
> ```
>
> As discussed on slack, it is hard to figure out an exact reason as to why
> the native JNI call fails with an invalid argument. Hence this PR proposes
> that the `lastLogMark` is updated to journal EOF instead of an arbitrary
> LONG.MAX_VALUE. The FileChannel interface defines that the implementors can
> pass in any long offset and the file handler should return EOF immediately
> when trying to read it. However it doesn't seem to be working as expected.
>
> ### Changes
>
> Updated `Journal#setLastLogMark()` method to accept an `scanOffset` instead
> of constant `LONG.MAX_VALUE`.
>
> ivankelly eolivelli
>
> Reviewers: Ivan Kelly , Enrico Olivelli <
> eolive...@gmail.com>
>
> This closes #2105 from karanmehta93/master
>
> Want to know about upcoming build environment updates?
>
> Would you like to stay up-to-date with the upcoming Travis CI build
> environment updates? We set up a mailing list for you!
> SIGN UP HERE <http://eepurl.com/9OCsP>
>
> [image: book icon]
>
> Documentation <https://docs.travis-ci.com/> about Travis CI
> Have any questions? We're here to help. 
> Unsubscribe
> <https://travis-ci.org/account/preferences/unsubscribe?repository=69758_medium=notification_source=email>
> from build emails from the apache/bookkeeper repository.
> To unsubscribe from *all* build emails, please update your settings
> <https://travis-ci.org/account/preferences/unsubscribe?utm_medium=notification_source=email>.
>
> [image: black and white travis ci logo] <https://travis-ci.com>
>
> Travis CI GmbH, Rigaer Str. 8, 10427 Berlin, Germany | GF/CEO: Randy Jacops
> | Contact: cont...@travis-ci.com | Amtsgericht Charlottenburg, Berlin, HRB
> 140133 B | Umsatzsteuer-ID gemäß §27 a Umsatzsteuergesetz: DE282002648


Re: Changing ledger metadata to binary format

2019-06-03 Thread Ivan Kelly
On Thu, May 30, 2019 at 12:15 AM Venkateswara Rao Jujjuri
 wrote:
>
> > "Let's decide this when we need it"
>
> Right. This is the time as we are trying to add fault domain info to the
> cookie.

When you say cookie, you mean under /ledgers/cookies? That falls
outside the scope of binary metadata.

> > to make a flag in the metadata store that we flip to enable it.
> Right I do recall the same. But given your proposal in this thread, all
> readers try to read in binary format,
> if that fails they fail back to text mode.

Actually, it doesn't even need this fallback, as the version of the
metadata is stored at the very start, so we switch on that.

> If this is the case, does the following make sense?
> 1. Bookies are updated first with new version. Replication workers
> recognize binary/text format and any metadata updates happen in the same
> format.
> 2. Clients are rolled out later with a caveat that the ledgers created by
> new clients can't be read by old clients. This may be ok as long as they
> don't share
> the same ledgers.
> With these two things, do we need an explicit flip to enable?

This works, but you need to expose a mechanism in the client to tell
it to write in V3.

As long as old clients don't try and read, there should be no problem.

> I can read the patches again, but let me ask :) did we take care of bookie
> shell also to understand binary format?

Yes, bookie shell can understand it since it reads though the same interfaces.

-Ivan


Re: Changing ledger metadata to binary format

2019-05-29 Thread Ivan Kelly
> What is our plan to move forward with binary format?

I've not plans regarding it. Moving forward with it will happen when
someone comes with a metadata change which will break text metadata
users (i.e. almost any metadata change).

> Anyone using binary format in production? even for new clusters?

We are not.

> What is blocking us from using binary format for the new clusters?

We need a tool to enable it. I can't remember the exact details, but
there was a conversation about whether to make each users specify the
metadata version in their configuration, or to make a flag in the
metadata store that we flip to enable it. I don't think we settled on
anything, and it turned into a "Let's decide this when we need it"
type of thing. The important thing at the time was to make it so
clients can _read_ binary metadata, so that these clients can then
read future updates.

-Ivan


Re: [VOTE] Release 4.9.1, release candidate #0

2019-04-04 Thread Ivan Kelly
Sorry about the delay on getting to this, Enrico.

+1 (binding)

 ✓ tag matches sha
 ✓ check sha512
 ✓ check gpg
 ✓ test licenses
 ✓ rat check
 ✓ compile -src
 ✓ spotbugs
 ✓ build tag
 ✓ compiled tag matches -src
 ✓ maven artifacts sigs are correct
 ✓ unit tests
 ✓ integration tests

A few minor non-blocking things:
- The tag isn't signed which may be a good thing to add to the process.
- Building from -src, results in 2 maven artifacts that are not pushed to nexus,
  * 
./org/apache/bookkeeper/bookkeeper-server/4.9.1/bookkeeper-server-4.9.1-javadoc.jar
  * 
./org/apache/bookkeeper/stats/bookkeeper-stats-api/4.9.1/bookkeeper-stats-api-4.9.1-javadoc.jar

-Ivan


[ANNOUNCE] Apache BookKeeper 4.8.2 released

2019-04-02 Thread Ivan Kelly
The Apache BookKeeper team is proud to announce Apache BookKeeper version
4.8.2.

Apache BookKeeper is a scalable, fault-tolerant, and low-latency
storage service optimized for
real-time workloads. It has been used for a fundamental service to
build reliable services.
It is also the log segment store for Apache DistributedLog and the
message store for Apache Pulsar.

This is the 17th release of the Apache BookKeeper.

The 4.8.2 release is a bugfix release which fixes a bunch of issues
reported from users of 4.8.1.

For BookKeeper release details and downloads, visit:

http://bookkeeper.apache.org/releases/

BookKeeper 4.8.2 Release Notes are at:

http://bookkeeper.apache.org/docs/4.8.2/overview/releaseNotes/

We would like to thank the contributors that made the release possible.

Regards,

The BookKeeper Team


Re: Long poll LAC returning immediately on fenced

2019-03-29 Thread Ivan Kelly
> Does the LAC change on that bookie if ledger is fenced?

It can do. The recovering client may write entries to that bookie
which hadn't originally arrived, but had arrived at another bookie.

-Ivan


Long poll LAC returning immediately on fenced

2019-03-29 Thread Ivan Kelly
Hi folks,

I'm seeing a problem where a bookie is getting hammer by long poll
requests. ~8000rps. This seems to be happening because the long poll
logic doesn't wait if the ledger is in fenced state [1], but returns
immediately. So the client ends up in a tight loop if the ledger has
entered fenced state and doesn't exit (I haven't found the root cause
for this yet).

Does anyone know why this check for fenced is there? LAC can change
during a recovery op, so it still makes sense to wait for the timeout.

Cheers,
Ivan


[1] 
https://github.com/apache/bookkeeper/blob/e3d807a32a0a9b69d0ac8db3ca17398373dbee28/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/FileInfo.java#L140


Re: Tags, version numbers and docker

2019-03-28 Thread Ivan Kelly
To clarify, what I intend to do here is.

1. Merge change from [1] to all live branches, so docker image will
build directly from the voted on tag.
2. Get infra to add another autobuild rule that matches on
release-([0-9.]+)\/docker, so that 4.8.2 and maybe 4.9.1 can be built
with this change. This rule would not be used after that point.

-Ivan

[1] 
https://github.com/ivankelly/bookkeeper/commit/e247ef705f055706604ba2f862c1006a8cf817e9

On Wed, Mar 27, 2019 at 10:52 AM Ivan Kelly  wrote:
>
> > That’s a known issue. The auto build is controlled by ASF. We have
> > discussed that before and came up the conclusion of current approach. There
> > is a BP to move dockerfile to a different repo. It just need someone to
> > complete the BP.
>
> This was a known issue a year ago. Nothing has moved on it, so I'm
> trying to make movement now.
>
> > If you did so, you will not release 4.8.2 image, no?
>
> The image isn't part of the official release. At most it's a
> convenience binary of the official release. However, at present it
> isn't even that since it's generated from a tag which has not been +1d
> by 3 PMC members.
>
> > Instead of doing a different way at the last phase of releasing a release,
> > I would suggest following the guide that was agreed by the community, and
> > work on the BP to move the dockerfile to a different repo in next release.
>
> There's a fundamental release issue here. The official release is the
> source tarball. The tag should reflect the contents of the source
> tarball, and it should be possible to generate all binary convenience
> packages from the source tarball.
>
> Things that do not match this criteria should not be presented as part
> of the release as they have not been approved by 3 members of the PMC.
>
> I'm not going to put my signature on a tag which hasn't been voted on.
> Currently whether tags constitute an official release artifact is
> unresolved from a legal POV [1], but they are two clicks away from the
> bookkeeper website frontpage so I think we should treat them as such.
>
> I will however fix the process for subsequent releases. The simplest
> fix is to set the BK_VERSION from tag name. This can be done in a
> build hook [2] which dockerhub autobuild[3] will pick up. Really
> though we should build the tarball in a prehook so that the image can
> also be generated for release candidates, so it can be tested and
> voted on along with the rest of the convenience binaries.
>
> -Ivan
>
> [1] https://issues.apache.org/jira/browse/LEGAL-438
> [2] 
> https://github.com/ivankelly/bookkeeper/commit/e247ef705f055706604ba2f862c1006a8cf817e9
> [3] 
> https://cloud.docker.com/repository/registry-1.docker.io/ivankelly/bookkeeper/builds


Re: Tags, version numbers and docker

2019-03-27 Thread Ivan Kelly
> That’s a known issue. The auto build is controlled by ASF. We have
> discussed that before and came up the conclusion of current approach. There
> is a BP to move dockerfile to a different repo. It just need someone to
> complete the BP.

This was a known issue a year ago. Nothing has moved on it, so I'm
trying to make movement now.

> If you did so, you will not release 4.8.2 image, no?

The image isn't part of the official release. At most it's a
convenience binary of the official release. However, at present it
isn't even that since it's generated from a tag which has not been +1d
by 3 PMC members.

> Instead of doing a different way at the last phase of releasing a release,
> I would suggest following the guide that was agreed by the community, and
> work on the BP to move the dockerfile to a different repo in next release.

There's a fundamental release issue here. The official release is the
source tarball. The tag should reflect the contents of the source
tarball, and it should be possible to generate all binary convenience
packages from the source tarball.

Things that do not match this criteria should not be presented as part
of the release as they have not been approved by 3 members of the PMC.

I'm not going to put my signature on a tag which hasn't been voted on.
Currently whether tags constitute an official release artifact is
unresolved from a legal POV [1], but they are two clicks away from the
bookkeeper website frontpage so I think we should treat them as such.

I will however fix the process for subsequent releases. The simplest
fix is to set the BK_VERSION from tag name. This can be done in a
build hook [2] which dockerhub autobuild[3] will pick up. Really
though we should build the tarball in a prehook so that the image can
also be generated for release candidates, so it can be tested and
voted on along with the rest of the convenience binaries.

-Ivan

[1] https://issues.apache.org/jira/browse/LEGAL-438
[2] 
https://github.com/ivankelly/bookkeeper/commit/e247ef705f055706604ba2f862c1006a8cf817e9
[3] 
https://cloud.docker.com/repository/registry-1.docker.io/ivankelly/bookkeeper/builds


Tags, version numbers and docker

2019-03-26 Thread Ivan Kelly
Hi folks,

Looking at doing the final tasks for the 4.8.2 release and stuck on
the docker bit. It's not that I don't see what has been done before,
but more that what is there is so so wrong.

Take 4.8.1 release for example. The tarball for that release was cut
from b4a2b1, yet the tag for that release is bf6149. bf6149 doesn't
exist on branch-4.8.

Instead, after b4a2b1 is cb8b76 which reverts 4.8.1 to 4.8.1-SNAPSHOT.
So versions are going backwards in the branch.

My bigger concern for now is the docker + tag bit. The tag should
represent exactly what we released as artifacts.

Who has control of the docker autobuild settings?  From reading the
docks, you can set a regex and use a capture group to set the version.
So for docker we could have additional tags, docker/release-4.8.2,
which lives on the branch-4.8. Another alternative is to move the
dockerfile into a different repo.

Anyhow, for now I'm going to set the tag to what was actually released.

-Ivan


Re: [VOTE] Release 4.8.2, release candidate #0

2019-03-14 Thread Ivan Kelly
With 3 +1 (binding), [Sijie, enrico and me], and no -1 the vote passes.

I'll follow up the rest of the process later today.

Thanks folks,
Ivan

On Thu, Mar 14, 2019 at 11:59 AM Ivan Kelly  wrote:
>
> +1 (binding from me too)
>
> - Licenses are good
> - rat, unit tests pass (some issues with checkstyle, but not going to
> let that block anything)
> - sig & sha512 good
>
> -Ivan
>
> On Wed, Mar 13, 2019 at 11:05 AM Enrico Olivelli  wrote:
> >
> > +1 (binding)
> > run tests on Fedora + java 8 (some known flaky test faiiled, not
> > blocker for this release)
> > checked signatures and checksums
> > run smoke tests with java 8 and localbookie and the bookkeeper shell
> >
> > Thank you Ivan
> > #shipit
> >
> > Enrico
> >
> > Il giorno lun 11 mar 2019 alle ore 09:05 Sijie Guo
> >  ha scritto:
> > >
> > > +1 (binding)
> > >
> > > - verified source & binary package
> > > - asc & sha512 are good
> > > - artifacts are good
> > > - tag is good
> > >
> > > On Sat, Mar 9, 2019 at 2:28 AM Ivan Kelly  wrote:
> > >
> > > > Hi everyone,
> > > > Please review and vote on the release candidate #0 for the version
> > > > 4.8.2, as follows:
> > > > [ ] +1, Approve the release
> > > > [ ] -1, Do not approve the release (please provide specific comments)
> > > >
> > > > The complete staging area is available for your review, which includes:
> > > > * Release notes [1]
> > > > * The official Apache source and binary distributions to be deployed
> > > > to dist.apache.org [2]
> > > > * All artifacts to be deployed to the Maven Central Repository [3]
> > > > * Source code tag "v4.8.2-rc0" [4] with git sha
> > > > f6adb282f88ab56afa3ef33b15d7acaf4b307eef
> > > >
> > > > BookKeeper's KEYS file contains PGP keys we used to sign this release:
> > > > https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
> > > >
> > > > Please download these packages and review this release candidate:
> > > >
> > > > - Review release notes
> > > > - Download the source package (verify shasum, and asc) and follow the
> > > > instructions to build and run the bookkeeper service.
> > > > - Download the binary package (verify shasum, and asc) and follow the
> > > > instructions to run the bookkeeper service.
> > > > - Review maven repo, release tag, licenses, and any other things you 
> > > > think
> > > > it is important to a release.
> > > >
> > > > The vote will be open for at least 72 hours. It is adopted by majority
> > > > approval, with at least 3 PMC affirmative votes. Since it's late on
> > > > friday, I won't tally the votes until this 17:00 UTC on wednesday
> > > > 13th.
> > > >
> > > > Have a great weekend folks,
> > > >
> > > > Ivan
> > > >
> > > > [1] https://github.com/apache/bookkeeper/pull/1976
> > > > [2]
> > > > https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.8.2-rc0/
> > > > [3]
> > > > https://repository.apache.org/content/repositories/orgapachebookkeeper-1039/
> > > > [4] https://github.com/apache/bookkeeper/tree/v4.8.2-rc0
> > > >


Re: [VOTE] Release 4.8.2, release candidate #0

2019-03-14 Thread Ivan Kelly
+1 (binding from me too)

- Licenses are good
- rat, unit tests pass (some issues with checkstyle, but not going to
let that block anything)
- sig & sha512 good

-Ivan

On Wed, Mar 13, 2019 at 11:05 AM Enrico Olivelli  wrote:
>
> +1 (binding)
> run tests on Fedora + java 8 (some known flaky test faiiled, not
> blocker for this release)
> checked signatures and checksums
> run smoke tests with java 8 and localbookie and the bookkeeper shell
>
> Thank you Ivan
> #shipit
>
> Enrico
>
> Il giorno lun 11 mar 2019 alle ore 09:05 Sijie Guo
>  ha scritto:
> >
> > +1 (binding)
> >
> > - verified source & binary package
> > - asc & sha512 are good
> > - artifacts are good
> > - tag is good
> >
> > On Sat, Mar 9, 2019 at 2:28 AM Ivan Kelly  wrote:
> >
> > > Hi everyone,
> > > Please review and vote on the release candidate #0 for the version
> > > 4.8.2, as follows:
> > > [ ] +1, Approve the release
> > > [ ] -1, Do not approve the release (please provide specific comments)
> > >
> > > The complete staging area is available for your review, which includes:
> > > * Release notes [1]
> > > * The official Apache source and binary distributions to be deployed
> > > to dist.apache.org [2]
> > > * All artifacts to be deployed to the Maven Central Repository [3]
> > > * Source code tag "v4.8.2-rc0" [4] with git sha
> > > f6adb282f88ab56afa3ef33b15d7acaf4b307eef
> > >
> > > BookKeeper's KEYS file contains PGP keys we used to sign this release:
> > > https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
> > >
> > > Please download these packages and review this release candidate:
> > >
> > > - Review release notes
> > > - Download the source package (verify shasum, and asc) and follow the
> > > instructions to build and run the bookkeeper service.
> > > - Download the binary package (verify shasum, and asc) and follow the
> > > instructions to run the bookkeeper service.
> > > - Review maven repo, release tag, licenses, and any other things you think
> > > it is important to a release.
> > >
> > > The vote will be open for at least 72 hours. It is adopted by majority
> > > approval, with at least 3 PMC affirmative votes. Since it's late on
> > > friday, I won't tally the votes until this 17:00 UTC on wednesday
> > > 13th.
> > >
> > > Have a great weekend folks,
> > >
> > > Ivan
> > >
> > > [1] https://github.com/apache/bookkeeper/pull/1976
> > > [2]
> > > https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.8.2-rc0/
> > > [3]
> > > https://repository.apache.org/content/repositories/orgapachebookkeeper-1039/
> > > [4] https://github.com/apache/bookkeeper/tree/v4.8.2-rc0
> > >


Re: Cutting 4.9.1

2019-03-13 Thread Ivan Kelly
+1 from me.

On Wed, Mar 13, 2019 at 10:16 AM Enrico Olivelli  wrote:
>
> Hi guys,
> I need to cut 4.9.1 because we are seeing very often this issue
> https://github.com/apache/bookkeeper/commit/25c7506c0513351c533db643cb10c953d1e6d0b7
>
> Please tag any issue you want to merge into 4.9 branch.
>
> I will start the release process hopefully within the end of the week
>
> Please remember that Ivan started a VOTE thread for the 4.8 branch as well
>
> Cheers
> Enrico


[VOTE] Release 4.8.2, release candidate #0

2019-03-08 Thread Ivan Kelly
Hi everyone,
Please review and vote on the release candidate #0 for the version
4.8.2, as follows:
[ ] +1, Approve the release
[ ] -1, Do not approve the release (please provide specific comments)

The complete staging area is available for your review, which includes:
* Release notes [1]
* The official Apache source and binary distributions to be deployed
to dist.apache.org [2]
* All artifacts to be deployed to the Maven Central Repository [3]
* Source code tag "v4.8.2-rc0" [4] with git sha
f6adb282f88ab56afa3ef33b15d7acaf4b307eef

BookKeeper's KEYS file contains PGP keys we used to sign this release:
https://dist.apache.org/repos/dist/release/bookkeeper/KEYS

Please download these packages and review this release candidate:

- Review release notes
- Download the source package (verify shasum, and asc) and follow the
instructions to build and run the bookkeeper service.
- Download the binary package (verify shasum, and asc) and follow the
instructions to run the bookkeeper service.
- Review maven repo, release tag, licenses, and any other things you think
it is important to a release.

The vote will be open for at least 72 hours. It is adopted by majority
approval, with at least 3 PMC affirmative votes. Since it's late on
friday, I won't tally the votes until this 17:00 UTC on wednesday
13th.

Have a great weekend folks,

Ivan

[1] https://github.com/apache/bookkeeper/pull/1976
[2] https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.8.2-rc0/
[3] https://repository.apache.org/content/repositories/orgapachebookkeeper-1039/
[4] https://github.com/apache/bookkeeper/tree/v4.8.2-rc0


Re: [VOTE] Release 4.9.0, release candidate #1

2019-01-30 Thread Ivan Kelly
+1 (binding)

(Ubuntu 16.06)
- Sigs: GOOD
- Licenses: GOOD
- Binary packages boot?: Tested starting bookie with -server & -all
packages. running bkctl simpletest against them. GOOD
- checkstyle: GOOD
- spotbugs: GOOD
- rat: GOOD
- test: GOOD
- integration test: GOOD
- Tested against pulsar master (unit tests, and selected integration
tests): GOOD

= Minor issues =
I spotted a view minor problems, so I created github issues for them.

zkServers & metadataServerUri are both specified in the config file by
default #1920
bkctl is shipping bin/bookkeeper #1921
bkctl simpletest fails silently #1922
src package is missing stream/clients/python/scripts/docker_build.sh #1923
standalone integration tests shouldn't run if -Dstream profile is omitted #1924

Good work Sijie!

-Ivan

On Wed, Jan 30, 2019 at 12:22 AM Matteo Merli  wrote:
>
> +1 (binding)
>
> (MacOS 10.14)
> Verified:
>   * Signatures
>   * Source package build
>   * Binary with localbookie
>   * Client library jars in Maven
>
>
> --
> Matteo Merli
> 
> On Mon, Jan 28, 2019 at 10:21 PM Jia Zhai  wrote:
> >
> > +1 (binding)
> >
> > Environment: macOS 10.14.2
> >
> >
> > - verified 4 packages checksum ( asc and sha all good)
> >
> > - the source package build and test all run successfully.
> >
> > - in both binary package(server & all), 'bin/bookkeeper standalone' and
> > 'bin/bookkeeper shell bookiesanity' runs well.
> >
> >
> >
> > On Tue, Jan 29, 2019 at 12:26 AM Enrico Olivelli 
> > wrote:
> >
> > > +1 (binding)
> > >
> > > * Run tests on jdk8 + linux, all passed.
> > > * Checked signatures, all okay
> > > * Run localbookie and "standalone mode" (without testing storage service)
> > > * Used with a downstream project and all tests are okay, but the
> > > migration from 8.1 was a "little" painful but doable:
> > > - old LedgerMetadata was repleaced with o.a.b.api.LedgerMetadata
> > > - changes in EnsemblePlacementPolicy (expected, we are braking that
> > > interface at every major version)
> > > - TestStatsProvider moved to bookkeeper-common:tests artifact, I had
> > > to fix a bunch of poms
> > >
> > > I have created a bunch of issues about running the bookie on JDK11,
> > > not blocker for a release.
> > > I have left a comment on the release notes PR, not blocker for a release.
> > >
> > > Thank you Sijie for putting all together !
> > >
> > > Enrico
> > >
> > > Il giorno dom 27 gen 2019 alle ore 05:38 Sijie Guo 
> > > ha scritto:
> > > >
> > > > Hi everyone,
> > > >
> > > > Please review and vote on the release candidate #1 for the version 
> > > > 4.9.0,
> > > > as follows:
> > > > [ ] +1, Approve the release
> > > > [ ] -1, Do not approve the release (please provide specific comments)
> > > >
> > > > The complete staging area is available for your review, which includes:
> > > > * Release notes [1]
> > > > * The official Apache source and binary distributions to be deployed to
> > > > dist.apache.org [2]
> > > > * All artifacts to be deployed to the Maven Central Repository [3]
> > > > * Source code tag "v4.9.0-rc1" [4] with git sha
> > > > b0e3adfea04e7e65512cca54e8b63d197abf910c
> > > >
> > > > BookKeeper's KEYS file contains PGP keys we used to sign this release:
> > > > https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
> > > >
> > > > Please download these packages and review this release candidate:
> > > >
> > > > - Review release notes
> > > > - Download the source package (verify shasum, and asc) and follow the
> > > > instructions to build and run the bookkeeper service.
> > > > - Download the binary package (verify shasum, and asc) and follow the
> > > > instructions to run the bookkeeper service.
> > > > - Review maven repo, release tag, licenses, and any other things you
> > > think
> > > > it is important to a release.
> > > >
> > > > The vote will be open for at least 72 hours. It is adopted by majority
> > > > approval, with at least 3 PMC affirmative votes.
> > > >
> > > > Thanks,
> > > > Release Manager
> > > >
> > > > [1] https://github.com/apache/bookkeeper/pull/1910
> > > > [2]
> > > https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.9.0-rc1/
> > > > [3]
> > > >
> > > https://repository.apache.org/content/repositories/orgapachebookkeeper-1038/
> > > > [4] https://github.com/apache/bookkeeper/tree/v4.9.0-rc1
> > >


Re: EnsemblePlacementPolicy exposes third party API "Pair" from commons-lang3 in a public API

2019-01-23 Thread Ivan Kelly
> > ```
> > class PlacementResult {
> > T result();
> > boolean strictlyConformsToPolicy();
> > }
>
> That was my first proposal and I like it. It is clearer and auto-documenting.
>
> Given that we are changing EnsemblePlacementPolicy at every major
> version, we can defer this refactor to 4.10.
> I am open to adopt this solution as well.

It's a small change, so better to apply it now. If you don't have the
cycles today, I can pick it up. We should merge your Pair stuff
anyhow. Pair is always useful.

-Ivan


Re: EnsemblePlacementPolicy exposes third party API "Pair" from commons-lang3 in a public API

2019-01-23 Thread Ivan Kelly
There's no harm in having our own tuple implementation in common, but
in this instance we should encode more meaning into the returned
value. As it is, it's not even java documented.
But in both cases, it looks like the boolean is whether the placement
strictly conforms to the placement policy, so both could return.

```
class PlacementResult {
T result();
boolean strictlyConformsToPolicy();
}
```

-Ivan

On Tue, Jan 22, 2019 at 6:55 PM Enrico Olivelli  wrote:
>
> Il mar 22 gen 2019, 18:38 Sijie Guo  ha scritto:
>
> > On Tue, Jan 22, 2019 at 8:40 AM Enrico Olivelli 
> > wrote:
> >
> > > Hi all,
> > > while reviewing 4.9 release I found this problem around a change about
> > > EnsemblePlacementPolicy
> > >
> > > this is the issue
> > > https://github.com/apache/bookkeeper/issues/1914
> > >
> > > The problem is that in public API we should not use third party
> > > classes in order to preserve compatibility with incompatible changes
> > > of the third party library.
> > >
> > > We already had such problems in the past.
> > >
> > > I think the best way to address this problem is to introduce one
> > > specific class in BookKeeper, maybe an inner class of
> > > EnsemblePlacementPolicy.
> > >
> >
> > why not just introduce a Pair like class in bookkeeper-common module?
> > instead of an inner class for EnsemblePlacementPolicy.
> >
>
> Works for me
>
> Enrico
>
> >
> >
> > >
> > > If we agree on this solution I can send the patch, it is very
> > > straightforward
> > >
> > > Regards
> > > Enrico
> > >
> >
> --
>
>
> -- Enrico Olivelli


Re: Clusterwide vs Client configuration for metadata format version

2018-12-19 Thread Ivan Kelly
> If it is client level configuration, in theory it is possible to have latest 
> client create v3 ledger while bookies are still running in the older version 
> right?

Yes, autorecovery would likely just break in this case.

> If we go with cluster level, I think using it part of LAYOUT_ZNODE is not
clean.
> I think we need to have a form of "cluster version number", or even better
with a combination of  capability/feature bit-map which can dictate
the cluster behavior.

I used the LAYOUT znode because that is what already exists. If we
create another znode for this, /ledgers/CLUSTER for example, then, for
consistency, the contents of the layout znode should really be moved
into this new znode. But this creates a lot more BC issues than just
using the LAYOUT znode. Old versions of the software ignore anything
other than the first two lines in LAYOUT. So, it's not clean nor
ideal, but it does work well within the constraints of BC.

> I am assuming that the tool Ivan is talking about is used for existing 
> clusters to update the cluster version number.
> Otherwise the maxLedgerMetadataFormat is used only for new clusters; that is 
> fine.

The maxLedgerMetadataFormat is only written when writing a new LAYOUT
node, so either during metaformat, or when using the proposed tool.
When it is absent from the layout node, it defaults to version 2,
which matches current behaviour.

The important thing for the 4.9 release is that the client can read
binary metadata, so that in 4.10 or 4.11, if we add a field to the
metadata, then we are able to use it with 4.9 clients and newer. It is
only that that point that maxLedgerMetadataFormat comes into play.

So, for the sake of getting 4.9 out the door, I propose that we:
a. Rollback the 2 changes around max metadata format version.
b. Pin serde to use V2 for now.
c. Continue this discussion to find the long term solution.

-Ivan


Re: Clusterwide vs Client configuration for metadata format version

2018-12-18 Thread Ivan Kelly
JV, Sam, Charan, Andrey, could one of you chime in on this? It's
holding up 4.9 release.

-Ivan

On Thu, Dec 13, 2018 at 5:38 PM Ivan Kelly  wrote:
>
> I'd be interested to see the opinion of the salesforce folks on this.
> On Thu, Dec 13, 2018 at 5:35 PM Ivan Kelly  wrote:
> >
> > > I am not sure about this. If clients don't react the changes of ledger
> > > layout,
> > > the information in ledger layout is just informative, you still need to
> > > coordinate
> > > both readers and writers. so IMO the version in ledger layout is not 
> > > really
> > > useful.
> >
> > The clients react the next time they initialize the ledger manager.
> > Which is exactly the same as would occur with a configuration setting.
> >
> > -Ivan


Re: Clusterwide vs Client configuration for metadata format version

2018-12-13 Thread Ivan Kelly
I'd be interested to see the opinion of the salesforce folks on this.
On Thu, Dec 13, 2018 at 5:35 PM Ivan Kelly  wrote:
>
> > I am not sure about this. If clients don't react the changes of ledger
> > layout,
> > the information in ledger layout is just informative, you still need to
> > coordinate
> > both readers and writers. so IMO the version in ledger layout is not really
> > useful.
>
> The clients react the next time they initialize the ledger manager.
> Which is exactly the same as would occur with a configuration setting.
>
> -Ivan


Re: Clusterwide vs Client configuration for metadata format version

2018-12-13 Thread Ivan Kelly
> I am not sure about this. If clients don't react the changes of ledger
> layout,
> the information in ledger layout is just informative, you still need to
> coordinate
> both readers and writers. so IMO the version in ledger layout is not really
> useful.

The clients react the next time they initialize the ledger manager.
Which is exactly the same as would occur with a configuration setting.

-Ivan


Re: Clusterwide vs Client configuration for metadata format version

2018-12-13 Thread Ivan Kelly
> I don't fully understand how the cluster-wide version work here, specially
> how do clients react when people use the tool to bump the version in ledger
> layout.

Clients don't have to react immediately. The cluster-wide setting is
the max _allowable_ format version. When it gets bumped, for example
from 2 to 3, clients that started when the value was 2 can continue to
write metadata in format 2, and all clients will will be able to read
it. Clients start after the bump can start to write in format 3. There
is currently nothing to motivate moving to version 3, but when we do
add something to the metadata protobuf, we will be able to have
clients read all the fields (even if it doesn't recognise it all).

> IMO a client setting is probably good enough and more flexible for people
> to control the upgrade stories and there will no surprises, since the
> version is controlled by the bookkeeper "writers".

I don't have a strong opinion either way. Client conf based gives more
power to users, but also requires more coordination among all users.
Clusterwide allows it to be set at a central authority, but that gives
users less freedom. Both have merits. How common is it for users from
different organisations to share ledgers?

-Ivan


Clusterwide vs Client configuration for metadata format version

2018-12-12 Thread Ivan Kelly
Hi folks,

A discussion has arisen about on [1] about the ledger layout changes
I've made recently [2].
The change[2] adds a field maxLedgerMetadataFormat to the cluster-wide
managed ledger layout.  When a new ledger is created, this is the
maximum format version which will be used to write it. Currently, the
default is 2 (text protobuf), though it could be changed to 3 (binary
protobuf) in future. I also had a plan to create a tool to bump to 3.
This field exists to allow clients that don't understand the binary
format to coexist with clients that do.
Another option would be to not change the ledger layout, but instead,
have a per client configuration for maxLedgerMetadataFormat, which
could default to 2. It would work the same way, but there would be no
central point to bump to 3. Each client/application would have to do
so.

What are folks thoughts on this? The cluster-wide is already
implemented, though per client has its advantages too. In any case,
this needs to be resolved before 4.9. Once resolved I'll push a BP for
the whole feature.

Cheers,
Ivan



[1] https://github.com/apache/bookkeeper/issues/1863
[2] https://github.com/apache/bookkeeper/pull/1858


Re: [VOTE] Release 4.7.3, release candidate #0

2018-12-04 Thread Ivan Kelly
+1 (binding)

* LICENSE & NOTICE look good.
* Rat good
* spotbugs good
* checkstyle had some issues, finding configs, but I'm not worried about it
* sha512 and gpg good
* tests ran cleanly
* ran pulsar master integration tests against it. all passed

Good work Sijie!
-Ivan
On Mon, Dec 3, 2018 at 8:19 PM Matteo Merli  wrote:
>
> +1
>
>  * Check binary, checksums, and ran a localbookie
>  * Source, checked rat and compiled with tests
>
> --
> Matteo Merli
> 
>
>
> On Fri, Nov 30, 2018 at 2:53 AM Sijie Guo  wrote:
>
> > Hi everyone,
> >
> > Please review and vote on the release candidate #0 for the version 4.7.3,
> > as follows:
> > [ ] +1, Approve the release
> > [ ] -1, Do not approve the release (please provide specific comments)
> >
> > The complete staging area is available for your review, which includes:
> > * Release notes [1]
> > * The official Apache source and binary distributions to be deployed to
> > dist.apache.org [2]
> > * All artifacts to be deployed to the Maven Central Repository [3]
> > * Source code tag "v4.7.3-rc0" [4] with git sha
> > 7a25ac301910271e80986e287db4595ae5f7785e
> >
> > BookKeeper's KEYS file contains PGP keys we used to sign this release:
> > https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
> >
> > Please download these packages and review this release candidate:
> >
> > - Review release notes
> > - Download the source package (verify shasum, and asc) and follow the
> > instructions to build and run the bookkeeper service.
> > - Download the binary package (verify shasum, and asc) and follow the
> > instructions to run the bookkeeper service.
> > - Review maven repo, release tag, licenses, and any other things you think
> > it is important to a release.
> >
> > The vote will be open for at least 72 hours. It is adopted by majority
> > approval, with at least 3 PMC affirmative votes.
> >
> > Thanks,
> > Sijie
> >
> > [1] https://github.com/apache/bookkeeper/pull/1850
> > [2]
> > https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.7.3-rc0/
> > [3]
> >
> > https://repository.apache.org/content/repositories/orgapachebookkeeper-1036/
> > [4] https://github.com/apache/bookkeeper/tree/v4.7.3-rc0
> >


Re: Bug with blockAddCompletions decrement

2018-12-04 Thread Ivan Kelly
> > Not an issue with master as blockAddCompletions has been replaced with
> > a simple boolean and failure handling changed to only deal with one
> > failure at a time. However, looking at it again, I think I did spot a
> > similar bug. will dig in after I send this.
>
> Great; looking forward.

It was something that sam had raised in the original patch. Fix is up.
https://github.com/apache/bookkeeper/pull/1857

> updateMetadataIfPossible() is called from resolveConflict() -> "return
> updateMetadataIfPossible(newMeta);"
> There is only one case in the  updateMetadataIfPossible(newMeta); that will
> rereadMetadata in other cases
> it can just return TRUE so there is no ChangeEnsembleCb.

When updateMetadataIfPossible is called, the precondition is that
there is at least one 'bump' that needs to be decremented on
blockAddCompletions before adds can be completed.
There's 3 branches in the code.
1. existing metadata is newer. reread is called, which will eventually
call updateMetadataIfPossible again under the same conditions
2. isConflictWith is true, so kill the ledger, we don't want to
decrement because we are in an undefined condition
3. we write new metadata, in which case ChangeEnsembleCb does the decrement.

This code is awful. It gives me nightmares. But AFAICS it is correct
w.r.t. to the preconditions stated above.

[1] 
https://github.com/apache/bookkeeper/blob/621799172a0de84f97b87d795c31e263de80e8a3/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java#L2197

-Ivan


Re: Bug with blockAddCompletions decrement

2018-12-03 Thread Ivan Kelly
> This may not be an issue with the master as we moved to immutable metadata,

Not an issue with master as blockAddCompletions has been replaced with
a simple boolean and failure handling changed to only deal with one
failure at a time. However, looking at it again, I think I did spot a
similar bug. will dig in after I send this.

Your email client mangled the code, but as far as I could untangle it.

updateMetadataIfPossible: No bug. decrement gets called when
ChangeEnsembleCb completes.

resolveConflict: Looks like a bug. Would be good to had a test case
that tickles it.

-Ivan


Re: Release 4.7.3

2018-11-29 Thread Ivan Kelly
What's the driving bugfix for cutting 4.7.3?
On Wed, Nov 21, 2018 at 11:54 PM Sijie Guo  wrote:
>
> Hi all,
>
> I would like to cut a 4.7.3 release late this week. If you have any fixes
> to include in 4.7.3, please label those issues as 4.7.3, then I will make
> sure they are included.
>
> - Sijie


Re: [VOTE] Release 4.8.1, release candidate #1

2018-11-21 Thread Ivan Kelly
+1(binding)

Ubuntu 16.04.5 LTS

- SHA512 & GPG good
- rat, spotbugs, tests good
- integration tests run cleanly
- licenses good

Good work Enrico!

-Ivan
On Wed, Nov 21, 2018 at 8:15 AM Jia Zhai  wrote:
>
> +1(binding).
>
> MacOS 10.14.1
>
> - SHA512 & GPG signatures good
> - local build, mvn test and integration test runs well
> - bookkeeper localbookie and bookiesanity runs well for all and server bin.
>
> On Mon, Nov 19, 2018 at 3:26 PM Enrico Olivelli  wrote:
>
> > +1 (binding)
> > checked binaries, run tests from source packages
> > run test of downstream projects
> > all on Linux (Fedora)
> >
> > We need at least 2 more PMCs votes.
> >
> > Any test and +1 from the community will be VERY appreciated
> >
> > Cheers
> > Enrico
> >
> > Il giorno gio 15 nov 2018 alle ore 13:55 Enrico Olivelli
> >  ha scritto:
> > >
> > > Hi everyone,
> > > Please review and vote on the release candidate #1 for the version
> > > 4.8.1, as follows:
> > > [ ] +1, Approve the release
> > > [ ] -1, Do not approve the release (please provide specific comments)
> > >
> > > The complete staging area is available for your review, which includes:
> > > * Release notes [1]
> > > * The official Apache source and binary distributions to be deployed
> > > to dist.apache.org [2]
> > > * All artifacts to be deployed to the Maven Central Repository [3]
> > > * Source code tag "v4.8.1-rc1" [4] with git sha
> > > b4a2b108301ca03437bf5247164e74c0249523c0
> > >
> > > BookKeeper's KEYS file contains PGP keys we used to sign this release:
> > > https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
> > >
> > > Please download these packages and review this release candidate:
> > >
> > > - Review release notes
> > > - Download the source package (verify shasum, and asc) and follow the
> > > instructions to build and run the bookkeeper service.
> > > - Download the binary package (verify shasum, and asc) and follow the
> > > instructions to run the bookkeeper service.
> > > - Review maven repo, release tag, licenses, and any other things you
> > think
> > > it is important to a release.
> > >
> > > The vote will be open for at least 72 hours. It is adopted by majority
> > > approval, with at least 3 PMC affirmative votes.
> > >
> > > Thanks,
> > > Enrico Olivelli
> > >
> > > [1] https://github.com/apache/bookkeeper/pull/1795
> > > [2]
> > https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.8.1-rc1/
> > > [3]
> > https://repository.apache.org/content/repositories/orgapachebookkeeper-1035/
> > > [4] https://github.com/apache/bookkeeper/tree/v4.8.1-rc1
> >


Re: Dropping 'stream' profile

2018-11-20 Thread Ivan Kelly
> Yes. That is a problem but it is not related stream profile. Let's separate
> unrelated issues into a different thread.
> Also I would suggest creating an issue in github when a problem is
> considered a bug, so the discussion can be more organized.

I'll create a bug.

> > I would suggest we also add a switch to enable the stream profile with
> > -Dall to harmonize with the package naming.
> >
>
> I am fine with a `-Dall`

I created a top level issue on github for sorting this out.  #1822

I have to get on a plane now, but I'll add a bunch of subtasks
tomorrow. Feel free to add anything I miss.

-Ivan


Re: Dropping 'stream' profile

2018-11-20 Thread Ivan Kelly
> There is no decision made. However I am -1 to drop stream profile, as I
> have explained in may different threads that I have been mentioned.

Where were these threads? I did a search in github and the list, but
couldn't see anything. i didn't even try slack, search there is awful.

> Here is the summary I don’t like dropping stream profile.
>
> I put table service and distributedlog under stream, because the whole
> stuff is built as an extension to bookkeeper core (ledger service). So this
> extension is optionally. With that being said, the bookkeeper distribution
> should can be built with and without stream profile.

So this is what is confusing and annoying about the profile. We say
it's optional, but then we only test with it on. We generate the
distributed binary packages with the profile. We only test end to end
functionality with the profile. So it's optional, but we always turn
the option on.

If we are to keep the profile, then the optional stuff should be truly
optional. I.e. server-bin with the profile should be identical to
server-bin without the flag. And all integration testing that isn't
testing functionality in the extensions should be against server-bin
without the profile. Anything that requires the profile in integration
testing should be in a different directory to make it clear that it's
not testing core.

I agree that the separation of core and extensions is good. But right
now, we're not actually enforcing it because this is an integration
problem, and we're only testing with the profile on.

> All the scripts is provided to be working at two conditions: in a released
> binary package and in a source directory. If it is running in a source
> directory and it is not built, it will run maven command to build and
> generating the classpath.

It did this, but when I typed yes, there was no indication that it was
doing this. It looked like the process hung, so i killed it, and only
later discovered the build had continued. I would consider this a bug.
Should be noisier.

I would suggest we also add a switch to enable the stream profile with
-Dall to harmonize with the package naming.

-Ivan


Re: Dropping 'stream' profile

2018-11-20 Thread Ivan Kelly
Resurrecting this thread, since it seems discusssions and decisions
have been made about it, but there's no record of these anywhere that
matters.

4.8.1-rc1 is out for vote, and once again if you build it in the
default way, random things do not work. Currently standalone doesn't
work straight away (and it kicks off a build in the background without
indicating whether anything is going on).

So, my questions are:
- Are we keeping -Dstream?
- If so, why?

And if we do, keep it, we need to disable anything in the default
build that depends on it.

I'll hold off from voting on the RC until there's a concrete plan to
resolve all this.

-Ivan
On Mon, Sep 17, 2018 at 10:11 AM Sijie Guo  wrote:
>
> Actually let's hold on this change for now.
>
> I was actually talking with JV about this. There are two many directories
> (modules) at the root level.
> It is a bit unclear about what modules. We might consider reorganizing the
> modules to make it clearer.
> I will come back with a proposal soon.
>
> - Sijie
>
> On Sun, Sep 16, 2018 at 5:18 AM Enrico Olivelli  wrote:
>
> > Sijie
> > This is a proposal
> > It is possilble that the precommit stuff will need a follow up patch, to
> > add a new precommit "subtask"
> >
> > Please check it out
> > https://github.com/apache/bookkeeper/pull/1680
> >
> > Enrico
> >
> > Il giorno lun 13 ago 2018 alle ore 09:57 Ivan Kelly  ha
> > scritto:
> >
> > > +1 for dropping the profiles.
> > >
> > > On Mon, Aug 13, 2018 at 12:24 AM, Sijie Guo  wrote:
> > > > I have no problem with this proposal. I am fine with dropping the
> > > profiles.
> > > >
> > > > Sijie
> > > >
> > > > On Sun, Aug 12, 2018 at 2:53 AM Enrico Olivelli 
> > > wrote:
> > > >
> > > >> Hi,
> > > >> Currently in order to build the full code you have to add -Dstream
> > > >> property, this in turn will activate the 'stream' profile.
> > > >> Additionally to run tests in 'stream' submodule you have to also add
> > > >> -DstreamTests.
> > > >>
> > > >> This is very annoying, and now that we are going to release the
> > 'stream'
> > > >> storage module as first class citizen it does not make much sense.
> > > >>
> > > >> This additional profile makes it more complex project wide operations
> > > like
> > > >> the release procedure.
> > > >> For instance I broke master branch yesterday because I did not advance
> > > the
> > > >> version in poms in the stream submodule.
> > > >>
> > > >> It is giving a lot of problems on code coverage stuff as well, because
> > > we
> > > >> have a very complex configuration of surefire.
> > > >>
> > > >> My proposal is to drop those profiles and let the stream module to be
> > > built
> > > >> together with the other parts.
> > > >>
> > > >>
> > > >> For the ones like me that work only on bookkeeper-server this change
> > > won't
> > > >> affect every day work.
> > > >>
> > > >> I would prefer that Sijie do this change as he introduced those
> > profiles
> > > >> and knowns very well all the tricks.
> > > >>
> > > >> Regards
> > > >> Enrico
> > > >> --
> > > >>
> > > >>
> > > >> -- Enrico Olivelli
> > > >>
> > >
> >


Re: Missing Apache BookKeeper 4.8.0 artifacts from Maven Central

2018-11-12 Thread Ivan Kelly
Oops, had searched for 4.8.0 and it looked like it was fresh in my
mailbox. my bad.

On Fri, Nov 9, 2018 at 5:46 PM, Sijie Guo  wrote:
> That was an email from Oct 4
>
> On Fri, Nov 9, 2018 at 5:01 AM Ivan Kelly  wrote:
>>
>> Wow. Suprised we're only hearing about it now
>>
>> -Ivan
>>
>> On Thu, Oct 4, 2018 at 1:14 PM, Enrico Olivelli 
>> wrote:
>> > Now it is okay
>> > https://search.maven.org/search?q=g:org.apache.bookkeeper
>> >
>> > Cheers
>> > Enrico
>> > Il giorno gio 4 ott 2018 alle ore 09:55 Enrico Olivelli
>> >  ha scritto:
>> >>
>> >> Hi,
>> >> During the release procedure of 4.8.0 something went wrong.
>> >> Apache Nexus Repositories did not publish artifacts to Maven Central.
>> >>
>> >> Artifacts will be available soon (I have simply "pushed the button"
>> >> another time on Nexus UI)
>> >>
>> >> Enrico


Re: [VOTE] Release 4.8.1, release candidate #0

2018-11-09 Thread Ivan Kelly
-1 (binding)

So in general the release looks fine, but there's still the error I
flagged with 4.8.0.

2018-11-09 16:18:43,271 - ERROR - [main:DockerUtils@188] -
DOCKER.exec(bookkeeper1_6e7e4ddf-717d-42bb-8701-4717d5027c92:/opt/bookkeeper/4.8.1/bin/bkctl
ledger simpletest --ensemble-size 3 --write-quorum-size 3
--ack-quorum-size 2 --num-entries 100): failed with 1 : Exception in
thread "main" java.util.ServiceConfigurationError:
org.apache.bookkeeper.tools.framework.CommandGroup: Provider
org.apache.bookkeeper.stream.cli.ClusterCommandGroup not found
at java.util.ServiceLoader.fail(ServiceLoader.java:239)
at java.util.ServiceLoader.access$300(ServiceLoader.java:185)
at 
java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:372)
at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
at org.apache.bookkeeper.tools.cli.BKCtl.main(BKCtl.java:52)

If you build without -Dstream bkctl is broken. If you build with  We
should either enable stream by default, or remove the bkctl dependency
on this class.

-Ivan

On Thu, Nov 8, 2018 at 10:24 AM, Enrico Olivelli  wrote:
> +1 (binding)
>
> - checked checksum and signatures of the staged artifacts on dist
> - checked RAT over the staged source artifacts
> - run unit tests on JDK8 on Linux (Fedora 27)
> - run tests on JDK10 over a few projects on Linux (Fedora 27)
> - run basic tests on JDK8 with localbookie and "standalone mode"
>
> Enrico
>
> Il giorno gio 8 nov 2018 alle ore 10:21 Enrico Olivelli
>  ha scritto:
>>
>> Hi everyone,
>> Please review and vote on the release candidate #0 for the version
>> 4.8.1, as follows:
>> [ ] +1, Approve the release
>> [ ] -1, Do not approve the release (please provide specific comments)
>>
>> The complete staging area is available for your review, which includes:
>> * Release notes [1]
>> * The official Apache source and binary distributions to be deployed
>> to dist.apache.org [2]
>> * All artifacts to be deployed to the Maven Central Repository [3]
>> * Source code tag "v4.8.1-rc0" [4] with git sha
>> 9f9f29f089958a13554056d901d293fad5614fa5
>>
>> BookKeeper's KEYS file contains PGP keys we used to sign this release:
>> https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
>>
>> Please download these packages and review this release candidate:
>>
>> - Review release notes
>> - Download the source package (verify shasum, and asc) and follow the
>> instructions to build and run the bookkeeper service.
>> - Download the binary package (verify shasum, and asc) and follow the
>> instructions to run the bookkeeper service.
>> - Review maven repo, release tag, licenses, and any other things you think
>> it is important to a release.
>>
>> The vote will be open for at least 72 hours. It is adopted by majority
>> approval, with at least 3 PMC affirmative votes.
>>
>> Thanks,
>> Enrico Olivelli
>>
>> [1] https://github.com/apache/bookkeeper/pull/1795
>> [2] https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.8.1-rc0/
>> [3] 
>> https://repository.apache.org/content/repositories/orgapachebookkeeper-1034/
>> [4] https://github.com/apache/bookkeeper/tree/v4.8.1-rc0


Re: Missing Apache BookKeeper 4.8.0 artifacts from Maven Central

2018-11-09 Thread Ivan Kelly
Wow. Suprised we're only hearing about it now

-Ivan

On Thu, Oct 4, 2018 at 1:14 PM, Enrico Olivelli  wrote:
> Now it is okay
> https://search.maven.org/search?q=g:org.apache.bookkeeper
>
> Cheers
> Enrico
> Il giorno gio 4 ott 2018 alle ore 09:55 Enrico Olivelli
>  ha scritto:
>>
>> Hi,
>> During the release procedure of 4.8.0 something went wrong.
>> Apache Nexus Repositories did not publish artifacts to Maven Central.
>>
>> Artifacts will be available soon (I have simply "pushed the button"
>> another time on Nexus UI)
>>
>> Enrico


Re: [VOTE] Release 4.8.0, release candidate #1

2018-09-24 Thread Ivan Kelly
Thanks for putting this together Enrico. I had left the test running
on friday and forgot to get back to it.

+1 from me (binding).

- Licenses good
- SHA512 & GPG signatures good
- Rat and spotbugs good
- mvn test runs cleanly without -Dstream & -DstreamTests
- integration tests run cleanly with -Dstream (the cli seems to depend
on something behind stream now)

The only negative is that with (-Dstream & -DstreamTests), there's a
test that hangs consistently
org.apache.bookkeeper.statelib.impl.kv.TestRocksdbKVAsyncStoreWithCheckpoints

I haven't looked into what this is, but i don't think this failure is
enough to block the release.

Cheers,
Ivan

On Thu, Sep 20, 2018 at 12:59 PM, Enrico Olivelli  wrote:
> Hi everyone,
> Please review and vote on the release candidate #1 for the version 4.8.0,
> as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
>
> The complete staging area is available for your review, which includes:
> * Release notes [1]
> * The official Apache source and binary distributions to be deployed to
> dist.apache.org [2]
> * All artifacts to be deployed to the Maven Central Repository [3]
> * Source code tag "release-4.8.0rc1" [4] with git sha
> 6172e31465865bd57290905a98b5a57e4c3956a4
>
> BookKeeper's KEYS file contains PGP keys we used to sign this release:
> https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
>
> Please download these packages and review this release candidate:
>
> - Review release notes
> - Download the source package (verify shasum, and asc) and follow the
> instructions to build and run the bookkeeper service.
> - Download the binary package (verify shasum, and asc) and follow the
> instructions to run the bookkeeper service.
> - Review maven repo, release tag, licenses, and any other things you think
> it is important to a release.
>
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
>
> Thanks,
> Enrico Olivelli
>
> [1] https://github.com/apache/bookkeeper/pull/1676
> [2] https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.8.0-rc1/
> [3]
> https://repository.apache.org/content/repositories/orgapachebookkeeper-1033/
> [4] https://github.com/apache/bookkeeper/tree/v4.8.0-rc1


Re: [VOTE] Release 4.8.0, release candidate #0

2018-09-18 Thread Ivan Kelly
It's as easy to do the fix as file an issue. Incoming.

-Ivan

On Tue, Sep 18, 2018 at 12:54 PM, Enrico Olivelli  wrote:
> Il giorno mar 18 set 2018 alle ore 11:41 Ivan Kelly  ha
> scritto:
>
>> Hey Enrico,
>>
>> Thanks for putting this together. Afraid it's -1 from me though.
>>
>> The new binaries pull in grpc, which has a notice file which we are
>> not bubbling up to our notice file.
>> https://github.com/grpc/grpc-java/blob/v1.12.0/NOTICE.txt
>>
>> There's also some minor issues with the links in the LICENSE file,
>> which makes it hard to check some dependencies.
>> - link for lib/org.apache.commons-commons-lang3-3.6.jar is wrong.
>> - link for lib/org.codehaus.jackson-jackson-core-asl-1.9.11.jar is
>> wrong (there's no version 1.9.11 there)
>> - likewise for lib/org.codehaus.jackson-jackson-mapper-asl-1.9.11.jar
>> - link for lib/com.google.api.grpc-proto-google-common-protos-1.0.0.jar is
>> wrong
>>
>> Otherwise, GPG sigs are good, SHA512 good. build seems to be good, but
>> hasn't finished for me yet.
>>
>
>
> Ivan,
> Thank you for your early feedback.
> Would you mind create an issue ?
> It should be stuff that went in with Stream Storage and maybe the same
> issues are also in 4.7.2
>
> Do you or Sijie have time to send a patch ? I am not sure I have cycles
> this week. Otherwise I will pick in up on the weekend
>
> I am canceling this RC by now.
>
> These issues are only about license files so code won't change in the next
> RC, any other feedback on this RC is welcome !!!
>
> Enrico
>
>
>
>>
>> -Ivan
>>
>>
>> On Sat, Sep 15, 2018 at 11:15 AM, Enrico Olivelli 
>> wrote:
>> > Ping
>> >
>> > Il mer 12 set 2018, 17:28 Enrico Olivelli  ha
>> scritto:
>> >
>> >> Hi everyone,
>> >> Please review and vote on the release candidate #0 for the version
>> 4.8.0,
>> >> as follows:
>> >> [ ] +1, Approve the release
>> >> [ ] -1, Do not approve the release (please provide specific comments)
>> >>
>> >> The complete staging area is available for your review, which includes:
>> >> * Release notes [1]
>> >> * The official Apache source and binary distributions to be deployed to
>> >> dist.apache.org [2]
>> >> * All artifacts to be deployed to the Maven Central Repository [3]
>> >> * Source code tag "release-4.8.0rc0" [4] with git sha
>> >> 6d4dd98696fa7c6b3f06067758cf79fd1137b611
>> >>
>> >> BookKeeper's KEYS file contains PGP keys we used to sign this release:
>> >> https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
>> >>
>> >> Please download these packages and review this release candidate:
>> >>
>> >> - Review release notes
>> >> - Download the source package (verify shasum, and asc) and follow the
>> >> instructions to build and run the bookkeeper service.
>> >> - Download the binary package (verify shasum, and asc) and follow the
>> >> instructions to run the bookkeeper service.
>> >> - Review maven repo, release tag, licenses, and any other things you
>> think
>> >> it is important to a release.
>> >>
>> >> The vote will be open for at least 72 hours. It is adopted by majority
>> >> approval, with at least 3 PMC affirmative votes.
>> >>
>> >> Thanks,
>> >> Enrico Olivelli
>> >>
>> >> [1] https://github.com/apache/bookkeeper/pull/1676
>> >> [2]
>> >> https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.8.0-rc0/
>> >> [3]
>> >>
>> https://repository.apache.org/content/repositories/orgapachebookkeeper-1032/
>> >> [4] https://github.com/apache/bookkeeper/tree/v4.8.0-rc0
>> >>
>> > --
>> >
>> >
>> > -- Enrico Olivelli
>>


Re: [VOTE] Apache BookKeeper Release 4.7.2, release candidate #0

2018-08-29 Thread Ivan Kelly
+1 (binding)

* Sigs good, checksums good
* Rat is good
* Licenses pass the automated check
* Tests pass
* Integration smoke test passes

-Ivan

On Tue, Aug 28, 2018 at 8:57 PM, Matteo Merli  wrote:
> +1 (binding)
>
>  * Checked signatures, LICENSES
>  * Checked Maven repository
>  * Started localbookie
>
>
> On Tue, Aug 28, 2018 at 11:54 AM Sijie Guo  wrote:
>
>> According to http://www.apache.org/dev/release-distribution#sigs-and-sums,
>> we need sha512 not sha1.
>>
>> Since it only changes the signatures files, I didn't cancel the vote.
>> Please review the sha512 files instead.
>>
>> - Sijie
>>
>> On Tue, Aug 28, 2018 at 8:03 AM Enrico Olivelli 
>> wrote:
>>
>> > +1 (binding)
>> > verified signatures/sha, RAT, spotbugs
>> > built and run tests from src candidate package (on Linux Fedora 27,
>> Oracle
>> > JDK8)
>> > tested with some client application
>> >
>> > I have these flaky tests (not blocker for release):
>> > the first BookieInitializationTest) is very usual on Fedora machines, I
>> > have already tracked the issue
>> > the second failure (TestHttpService) happens only if running the whole
>> > TestHttpService file, not the single test
>> > this is the issue https://github.com/apache/bookkeeper/issues/1630
>> >
>> > [ERROR] Failures:
>> > [ERROR]
>> >
>> >
>> org.apache.bookkeeper.bookie.BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile(org.apache.bookkeeper.bookie.BookieInitializationTest)
>> > [ERROR]   Run 1:
>> >
>> BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile:754
>> > Bookie should be up and running
>> > [ERROR]   Run 2:
>> >
>> BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile:754
>> > Bookie should be up and running
>> > [ERROR]   Run 3:
>> >
>> BookieInitializationTest.testWithDiskFullAndAbilityToCreateNewIndexFile:754
>> > Bookie should be up and running
>> > [INFO]
>> > [ERROR]
>> >
>> >
>> org.apache.bookkeeper.server.http.TestHttpService.testWhoIsAuditorService(org.apache.bookkeeper.server.http.TestHttpService)
>> > [ERROR]   Run 1: TestHttpService.testWhoIsAuditorService:638
>> expected:<200>
>> > but was:<404>
>> > [ERROR]   Run 2: TestHttpService.testWhoIsAuditorService:638
>> expected:<200>
>> > but was:<404>
>> > [ERROR]   Run 3: TestHttpService.testWhoIsAuditorService:638
>> expected:<200>
>> > but was:<404>
>> >
>> > Enrico
>> >
>> >
>> > Il giorno lun 27 ago 2018 alle ore 20:01 Sijie Guo 
>> ha
>> > scritto:
>> >
>> > > Enrico,
>> > >
>> > > I fixed the sha files.
>> > >
>> > > - Sijie
>> > >
>> > > On Mon, Aug 27, 2018 at 10:57 AM Sijie Guo  wrote:
>> > >
>> > > > Ah I see. I think the script wasn't fixed in branch-4.7. Let me
>> update
>> > > > those sha files.
>> > > >
>> > > > - Sijie
>> > > >
>> > > > On Mon, Aug 27, 2018 at 6:51 AM Enrico Olivelli > >
>> > > > wrote:
>> > > >
>> > > >> Sijie,
>> > > >> thank you for running the release
>> > > >>
>> > > >> the .sha files contains absolute paths
>> > > >> af1c5f1451fc02fb60bfe8eac91c0f5bd54895ad
>> > > >>
>> > > >>
>> > >
>> >
>> /Users/sijie/Workspaces/bookkeeper/target/dist_dev/bookkeeper/bookkeeper-4.7.2-rc0/bookkeeper-4.7.2-src.tar.gz
>> > > >> 30c6a6b75c3715572f2d37ff69d0b08b1dfcf6ac
>> > > >>
>> > > >>
>> > >
>> >
>> /Users/sijie/Workspaces/bookkeeper/target/dist_dev/bookkeeper/bookkeeper-4.7.2-rc0/bookkeeper-all-4.7.2-bin.tar.gz
>> > > >> 08d6cd1920e4d3f140911c4e71235bf27729abd5
>> > > >>
>> > > >>
>> > >
>> >
>> /Users/sijie/Workspaces/bookkeeper/target/dist_dev/bookkeeper/bookkeeper-4.7.2-rc0/bookkeeper-server-4.7.2-bin.tar.gz
>> > > >>
>> > > >> can you please fix them ?
>> > > >> then I will be able to send a (positive) vote
>> > > >>
>> > > >> Enrico
>> > > >>
>> > > >> Il giorno lun 27 ago 2018 alle ore 10:38 Sijie Guo <
>> > guosi...@gmail.com>
>> > > >> ha
>> > > >> scritto:
>> > > >>
>> > > >> > Hi everyone,
>> > > >> >
>> > > >> > Please review and vote on the release candidate #0 for the version
>> > > >> 4.7.2,
>> > > >> > as follows:
>> > > >> > [ ] +1, Approve the release
>> > > >> > [ ] -1, Do not approve the release (please provide specific
>> > comments)
>> > > >> >
>> > > >> > The complete staging area is available for your review, which
>> > > includes:
>> > > >> > * Release notes [1]
>> > > >> > * The official Apache source and binary distributions to be
>> deployed
>> > > to
>> > > >> > dist.apache.org [2]
>> > > >> > * All artifacts to be deployed to the Maven Central Repository [3]
>> > > >> > * Source code tag "v4.7.2-rc0" [4] with git
>> > > >> > sha ea072c8f8b9d20651b513036c09f264ca5c7ec6e
>> > > >> >
>> > > >> > BookKeeper's KEYS file contains PGP keys we used to sign this
>> > release:
>> > > >> > https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
>> > > >> >
>> > > >> > Please download these packages and review this release candidate:
>> > > >> >
>> > > >> > - Review release notes
>> > > >> > - Download the source package (verify shasum, and asc) and follow
>> > the
>> > > >> > instructions to build and run 

Re: OrderedScheduler & OrderedExecutor in bookkeeper client

2018-08-23 Thread Ivan Kelly
> In the case of OrderedExecutor, it needs a BlockingQueue and the current
> default is to use JDK LinkedBlockingQueue which relies on CAS for
> enqueue/dequeue. Additional room for improvement here is to use a more
> specialized MP-SC queue with different wait strategies.

+1 to this, though something to keep in mind here is that these aren't
BK's cores we're running on, they belong to the client application.

> Overall, I think we should still keep separated the critical data path from
> the rest of operations that are not necessarily time critical.

Keeping the scheduling threads and the executor threads separate means
that we need monitors, which can mean potential blocking, and other
threads invalidating the cpu cache of thread could potentially be kept
very warm.

>> What I want to achieve here is that, after a operation is submitted to
> a ledger, it only ever operates on a single thread.
>
> I think a viable option would be to keep the scheduler separate and then
> always jump in the ledger thread. eg:
>
> scheduler.schedule(() -> {
> executor.executeOrdered(ledgerId, SafeRunnable.safeRun(() -> {
> // do something
> });
> }, 100, TimeUnit.MILLISECONDS);
>

This is basically what I was suggesting, except that we hide this
behind the OrderedScheduler interface. Passing in both the scheduler
and the executor allows the user to run stuff on another thread
unwittingly.

-Ivan


Re: OrderedScheduler & OrderedExecutor in bookkeeper client

2018-08-23 Thread Ivan Kelly
> I don't think it is accidently. OrderedExecutor has performance advantages
> than OrderedScheduler.
>
> A bit background on this:
>
> - OrderedScheduler was introduced by me. And I changed existing
> OrderedSafeExecutor to be extending from OrderedScheduler.
>   Trying to standarize to one `OrderedScheduler`.
> - I think Matteo noticed performance difference between `executor` and
> `scheduler`, so he made the change
>
> https://github.com/apache/bookkeeper/commit/46171e67e526702487641438144f28b7eb1aa07b
> .
> So the `executor` is used as the main callback executor, since it just
> requires ordering but doesn't need scheduling capability.
> the `scheduler` is used for scheduling tasks but doesn't require ordering.

The scheduler does need ordering in one case for explicit Lac.

I think we could modify the scheduler, so that we could use the same
scheduler object for the executor and for scheduling.

Instead of having multiple executors in the scheduler, just create one
scheduled executor, which then submits to the executor service after
the delay.

What I want to achieve here is that, after a operation is submitted to
a ledger, it only ever operates on a single thread.
If you look at LedgerHandle now, you have to jump around 4 files to
deduce which thread methods like handleBookieFailure or
sendAddSuccessCallbacks are called on, and even then you can't even be
sure, so we wrap everything in synchronized when we don't really need
to.

-Ivan


Re: OrderedScheduler & OrderedExecutor in bookkeeper client

2018-08-23 Thread Ivan Kelly
>> We currently create an OrderedExecutor and an OrderedScheduler in the
>> client. An OrderedScheduler is an OrderedExecutor. Moreover, it's very
>> seldom used (basically for polling LAC, speculative reads and explicit
>> flush.
>
> Why do they exist? Are they only legacy from past or is there any specific
> reason?

Accident I think. Previously, OrderedExecutor didn't implement
scheduling, and BookKeeper had a ScheduledExecutorService member.
This changed with
https://github.com/apache/bookkeeper/commit/46171e67e526702487641438144f28b7eb1aa07b.

> Is there any per ledger ordering requirement that we will break (or cause
> deadlock) if we have a single queue per ledger?

Any ordering that depends on this is broken. No operations should
block on either of these executors, and I don't think any do.

-Ivan


OrderedScheduler & OrderedExecutor in bookkeeper client

2018-08-23 Thread Ivan Kelly
Hi folks,

We currently create an OrderedExecutor and an OrderedScheduler in the
client. An OrderedScheduler is an OrderedExecutor. Moreover, it's very
seldom used (basically for polling LAC, speculative reads and explicit
flush.

I propose that we fold these into one. i.e. construct an
OrderedScheduler, constructed with the same parameters as we build the
OrderedExecutor now, and use this OrderedScheduler for all cases
rather than the OrderedExecutor.

(this is the first in a bunch of changes I'll be proposing to clean up
the client thread model).

-Ivan


Re: [DISCUSS] BookKeeper 4.7.2 release

2018-08-16 Thread Ivan Kelly
> In a more concrete example, currently Pulsar 2.1.0 is using BK 4.7.1. There
> have been multiple issues reported from Pulsar users using 4.7.1.
> In order to address the problems facing by the users using BK 4.7.1, we
> have to create this 4.7.2 release for fixing the bugs in 4.7.1.

If it's needed for Pulsar, then +1 from me.

-Ivan


Re: [DISCUSS] BookKeeper 4.7.2 release

2018-08-16 Thread Ivan Kelly
What changes are in 4.8.0 that was prevent users from moving directly to that?

Is anyone requesting 4.7.2?

-Ivan

On Thu, Aug 16, 2018 at 8:59 AM, Sijie Guo  wrote:
> Hi all,
>
> There are bunch of fixes cherry-picked into branch-4.7 and some are marked
> for cherry-picking to branch-4.7. I think it is time to discuss cutting a
> bug fix release 4.7.2. Any thoughts?
>
> - Sijie


Re: Usefulness of ensemble change during recovery

2018-08-13 Thread Ivan Kelly
Yup, we had already concluded we need the ensemble change for some
cases. Code didn't turn out as messy as I'd feared though (I don't
think I've pushed this yet).

-Ivan

On Mon, Aug 13, 2018 at 8:29 PM, Sam Just  wrote:
> To flesh out JV's point a bit more, suppose we've got a 5/5/4 ledger which
> needs to be recovery opened.  In such a scenario, suppose the last entry on
> each of the 5 bookies (no holes) are 10,10,10,10,19.  Any entry in [10,19]
> is valid as the end of the ledger, but the safest answer for the end of the
> ledger is really 10 here -- 11-19 cannot have been ack'd to the client and
> we have 5 copies of 0-10, but only 1 of 11-19.  Currently, a client
> performing a recovery open on this ledger which is able to talk to all 5
> bookies will read and rewrite up to 19 ensuring that at least 4 bookies end
> up with 11-19.  I'd argue that rewriting the entries in that case is
> important if we want to let 19 be the end of the ledger because once we
> permit a client to read 19, losing that single copy would genuinely be data
> loss.  In that case, it happens that we have enough information to mark 10
> as the end of the ledger, but if the client performing recovery open has
> access only to bookies 3 and 4, it would be forced to conclude that 19
> could be the end of the ledger.  In that case, if we want to avoid exposing
> entries which have never been written to fewer than aQ bookies, we really
> do have to either
> 1) do an ensemble change and write out the tail entries of the ledger to a
> healthy ensemble
> 2) fail the recovery open
>
> I'd therefore argue that repairing the tail of the ledger -- with an
> ensemble change if necessary -- is actually required to allow readers to
> access the ledger.
> -Sam
>
> On Mon, Aug 6, 2018 at 9:27 AM Venkateswara Rao Jujjuri 
> wrote:
>
>> I don't think it a good idea to leave the tail to the replication.
>> This could lead to the perception of data loss, and it's more evident in
>> the case of larger WQ and disparity with AQ.
>> If we determine LLAC based on having 'a copy', which is never acknowledged
>> to the client, and if that bookie goes down(or crashes and burns)
>> before replication worker gets a chance, it gives the illusion of data
>> loss. Moreover, we have no way to determine the real data loss vs
>> this scenario where we have never acknowledged the client.
>>
>>
>> On Mon, Aug 6, 2018 at 12:32 AM, Sijie Guo  wrote:
>>
>> > On Mon, Aug 6, 2018 at 12:08 AM Ivan Kelly  wrote:
>> >
>> > > >> Recovery operates on a few seconds of data (from the last LAC
>> written
>> > > >> to the end of the ledger, call this LLAC).
>> > > >
>> > > > the data during this duration can be very large if the traffic of the
>> > > > ledger is large. That has
>> > > > been observed at Twitter's production. so when we are talking about
>> "a
>> > > few
>> > > > seconds of data",
>> > > > we can't assume the amount of data is little. That says the recovery
>> > can
>> > > be
>> > > > taking time than
>> > >
>> > > Yes, it can be large, but still it is only a few seconds worth of
>> > > data. It is the amount of data that can be transmitted in the period
>> > > of one roundtrip, as the next roundtrip will update the LAC.
>> >
>> >
>> > > I didn't mean to imply the data was small. I was implying that the
>> > > data was small in comparison to the overall size of that ledger.
>> >
>> >
>> > > > what we can expect, so if we don't handle failures during recovery
>> how
>> > we
>> > > > are able to ensure
>> > > > we have enough data copy during recovery.
>> > >
>> > > Consider a e3w3a2 ledger, there's two cases where you can lose a
>> > > bookie during recover.
>> > >
>> > > Case one, one bookie is lost. You can still recover from as ack=2 is
>> > > available.
>> > > Case two, two bookies are lost. You can't recover, but ledger is
>> > > unavailable anyhow, since any entry in the ledger may only have been
>> > > replicated to 2.
>> > >
>> > > However, with e3w3a3 I guess you wouldn't be able to recover at all,
>> > > and we have to handle that case.
>> > >
>> > > > I am not sure "make ledger metadata immutable" == "getting rid of
>> > merging
>> > > > ledger metadata".
>> > > > because I don't think these

Re: Dropping 'stream' profile

2018-08-13 Thread Ivan Kelly
+1 for dropping the profiles.

On Mon, Aug 13, 2018 at 12:24 AM, Sijie Guo  wrote:
> I have no problem with this proposal. I am fine with dropping the profiles.
>
> Sijie
>
> On Sun, Aug 12, 2018 at 2:53 AM Enrico Olivelli  wrote:
>
>> Hi,
>> Currently in order to build the full code you have to add -Dstream
>> property, this in turn will activate the 'stream' profile.
>> Additionally to run tests in 'stream' submodule you have to also add
>> -DstreamTests.
>>
>> This is very annoying, and now that we are going to release the 'stream'
>> storage module as first class citizen it does not make much sense.
>>
>> This additional profile makes it more complex project wide operations like
>> the release procedure.
>> For instance I broke master branch yesterday because I did not advance the
>> version in poms in the stream submodule.
>>
>> It is giving a lot of problems on code coverage stuff as well, because we
>> have a very complex configuration of surefire.
>>
>> My proposal is to drop those profiles and let the stream module to be built
>> together with the other parts.
>>
>>
>> For the ones like me that work only on bookkeeper-server this change won't
>> affect every day work.
>>
>> I would prefer that Sijie do this change as he introduced those profiles
>> and knowns very well all the tricks.
>>
>> Regards
>> Enrico
>> --
>>
>>
>> -- Enrico Olivelli
>>


Status of immutable metadata

2018-08-09 Thread Ivan Kelly
Hi folks,

As some of you are aware I've been working on making the client
metadata immutable. What this means is that the client will only act
on metadata that reflects what is in zookeeper.

I've pretty much got all the code done for it. However, the last 2
days I've been sidetracked by a bug I saw while working on the
metadata.

https://github.com/apache/bookkeeper/issues/1591

It's a pretty serious bug. The client can report data as persisted
that then disappears. And moving to immutable metadata doesn't solve
it. Given that the bug is in the same area as metadata updates, we
should resolve it before I push the final metadata changes.


-Ivan


Re: [Draft] ASF Board Report for BookKeeper (August)

2018-08-08 Thread Ivan Kelly
lgtm +1

On Wed, Aug 8, 2018 at 10:56 AM, Enrico Olivelli  wrote:
> Looks good, thank you Sijie for taking care of this.
>
> Enrico
>
> Il giorno mer 8 ago 2018 alle ore 10:48 Sijie Guo  ha
> scritto:
>
>> Hi all,
>>
>> We have a board report due today. Here is a draft. Please take a look and
>> let me know what you think.
>>
>> -
>>
>> ## Description:
>>
>> BookKeeper is a scalable, fault-tolerant, and low-latency storage
>> service optimized for append-only workloads. It has been used as
>> a fundamental service to build high available and replicated services
>> in companies like Twitter, Yahoo and Salesforce. It is also the log
>> segment store for Apache DistributedLog and message store for Apache
>> Pulsar.
>>
>> Apache DistributedLog is a high-level API and service layer for
>> Apache BookKeeper, providing easier access to the BookKeeper
>> primitives. It is a subproject of Apache BookKeeper.
>>
>> ## Issues:
>>
>> There are no issues requiring board attention at this time.
>>
>> ## Activity:
>>
>>  - 4.7.1 was released on Tue Jun 19 2018
>>  - 4.8.0 was scheduled to be release in August. It is delayed due to some
>> discussions around metadata upgrade.
>>  - Apache Pulsar (incubating) has fully integrated with the official
>> BookKeeper release 4.7.0.
>>The growth of Pulsar community also help grow the adoption of
>> BookKeeper.
>>  - Deep integration between Pulsar and BookKeeper, for example Pulsar
>> Functions integration with bookkeeper table service.
>>This helps building the ecosystem around BookKeeper.
>>
>> ## Health report:
>>
>>  - Development continues at a steady pace. We are merging multiple PRs per
>> day on average.
>>  - Mailing list and slack discussions are brisk, in particularly around the
>> active projects.
>>
>> ## PMC changes:
>>
>>  - Currently 15 PMC members.
>>  - No new PMC members added in the last 3 months
>>  - Last PMC addition was Enrico Olivelli on Thu Feb 22 2018
>>
>> ## Committer base changes:
>>
>>  - Currently 21 committers.
>>  - No new committers added in the last 3 months
>>  - Last committer addition was Andrey Yegorov at Fri Feb 09 2018
>>
>> ## Releases:
>>
>>  - 4.7.1 was released on Tue Jun 19 2018
>>
>> ## Mailing list activity:
>>
>>  - dev@bookkeeper.apache.org:
>> - 105 subscribers (up 1 in the last 3 months):
>> - 132 emails sent to list (327 in previous quarter)
>>
>>  - distributedlog-comm...@bookkeeper.apache.org:
>> - 12 subscribers (up 0 in the last 3 months):
>> - 2 emails sent to list (2 in previous quarter)
>>
>>  - distributedlog-...@bookkeeper.apache.org:
>> - 42 subscribers (up 0 in the last 3 months):
>> - 0 emails sent to list (0 in previous quarter)
>>
>>  - distributedlog-iss...@bookkeeper.apache.org:
>> - 9 subscribers (up 0 in the last 3 months):
>> - 2 emails sent to list (67 in previous quarter)
>>
>>  - distributedlog-u...@bookkeeper.apache.org:
>> - 26 subscribers (up 0 in the last 3 months):
>> - 0 emails sent to list (0 in previous quarter)
>>
>>  - iss...@bookkeeper.apache.org:
>> - 9 subscribers (up 0 in the last 3 months):
>> - 2088 emails sent to list (3044 in previous quarter)
>>
>>  - u...@bookkeeper.apache.org:
>> - 117 subscribers (up 2 in the last 3 months):
>> - 39 emails sent to list (41 in previous quarter)
>>
>> Thanks,
>> Sijie
>>


Re: Usefulness of ensemble change during recovery

2018-08-06 Thread Ivan Kelly
>> Recovery operates on a few seconds of data (from the last LAC written
>> to the end of the ledger, call this LLAC).
>
> the data during this duration can be very large if the traffic of the
> ledger is large. That has
> been observed at Twitter's production. so when we are talking about "a few
> seconds of data",
> we can't assume the amount of data is little. That says the recovery can be
> taking time than

Yes, it can be large, but still it is only a few seconds worth of
data. It is the amount of data that can be transmitted in the period
of one roundtrip, as the next roundtrip will update the LAC.

I didn't mean to imply the data was small. I was implying that the
data was small in comparison to the overall size of that ledger.

> what we can expect, so if we don't handle failures during recovery how we
> are able to ensure
> we have enough data copy during recovery.

Consider a e3w3a2 ledger, there's two cases where you can lose a
bookie during recover.

Case one, one bookie is lost. You can still recover from as ack=2 is available.
Case two, two bookies are lost. You can't recover, but ledger is
unavailable anyhow, since any entry in the ledger may only have been
replicated to 2.

However, with e3w3a3 I guess you wouldn't be able to recover at all,
and we have to handle that case.

> I am not sure "make ledger metadata immutable" == "getting rid of merging
> ledger metadata".
> because I don't think these are same thing. making ledger metadata
> immutable will make code
> much clearer and simpler because the ledger metadata is immutable. how
> getting rid of merging
> ledger metadata is a different thing, when you make ledger metadata
> immutable, it will help make
> merging ledger metadata on conflicts clearer.

I wouldn't call it merging in this case. Merging implies taking two
valid pieces of metadata and getting another usable, valid metadata
from it.
What happens with immutable metadata, is that you are taking one valid
metadata, and applying operations to it. So in the failure during
recovery place, we would have a list of AddEnsemble operations which
we add when we try to close.

In theory this is perfectly valid and clean. It just can look messy in
the code, due to how the PendingAddOp reaches back into the ledger
handle to get the current ensemble.

So, in conclusion, I will keep the handling. In any case, these
changes are all still blocked on
https://github.com/apache/bookkeeper/pull/1577.

-Ivan


Usefulness of ensemble change during recovery

2018-08-04 Thread Ivan Kelly
Hi folks,

Recently I've been working to make the ledger metadata on the client
immutable, with the goal of making client metadata management more
understandable. The basic idea is that the metadata the client uses
should reflect what is in zookeeper. So if a client wants to modify
the metadata, if makes a copy, modifies, writes to zookeeper and then
starts using it. This gets rid of all the confictsWith and merge
operations.

There is only one case where this doesn't work. When we recover a
ledger, we read the LAC from all bookies, then read forward entry by
entry, rewriting the entry, until we reach the end. If a bookie fails
during the rewrite, we replace it in the ensemble, but we don't write
that back to zookeeper until the end.

I was banging my head off this yesterday, trying to find a nice way to
fit this in (there's loads of nasty ways), when I came to the
conclusion that failure recovery during recovery isn't actually
useful.

Recovery operates on a few seconds of data (from the last LAC written
to the end of the ledger, call this LLAC). Take a ledger with 3:2:2
configuration. If the writer crashes, and one bookie crashes, when we
recover we currently replace that crashed bookie, so that if another
bookie crashes the data is still available. But, and this is why I
don't think it's useful, if another bookie crashes, the recovered data
may be available, but everything before the LLAC in the ledger will
not be available.
IMO, this kind of thing should be handled by rereplication, not
ensemble change (as as aside, we should have a hint system to trigger
rereplication ASAP on this ledger).

Anyhow, I'd like to hear other opinions on this before proceeding.
Recovery with ensemble changes can work. Rather than modifying the
ledger, create a shadow ensemble list, and give entries from that to
the writers, but with the current entanglement in the client, this is
a bit nasty.

Cheers,
Ivan


Re: Changing ledger metadata to binary format

2018-07-30 Thread Ivan Kelly
>> Thank you for putting this together. It is also good to put this as a BP,
>> since it is about the metadata layout.

I'll put a BP up this week once I have initial feedback.

>> > - When writing a metadata, check what is in /ledgers/LAYOUT. If it is
>> > as above, write using the current text protobuf. If is bumped, use the
>> > new binary format.
>
> isn't it too costly? Adding a zk read for each write. We could add a watch
> but is has an important cost

This cost can be amortized. In simplest case, we only read when client
starts (as already happens). Client are able both read and write the
old format, so clients that read the pre-upgrade version will still
write in text format, and this is fine.

However, we would want them to eventually see the new format without a
restart. At one extreme, we could put a watch on the znode, but this
could be quite heavy (I think the only per-client watch we have now is
on /ledgers/available). Alternatively, we could just read
/ledgers/LAYOUT once per day. This way clients who saw the old version
will eventually get updated.

> What about having a client side config writeMetadataVersion ?
> We start a new metadata version, the new one  will be encoded as binary.
> By default 4.8 clients will use previous version, as we already do for
> journal and fileinfo on bookies.
>
> Each ledger is independent from the others, there is no need of a global
> flag written on zk.

There is a global property here though,which only the administrator
can control. It is "Can all clients in the cluster/application read
the new ledger metadata version?". This is why I'd like to keep it as
a centralized flag rather than allowing each client to decide for
themselves.

> Don't we already have some tool? (Maybe I only have seen such tool in my
> company applications)

It may already exist, I haven't checked.

-Ivan


Changing ledger metadata to binary format

2018-07-27 Thread Ivan Kelly
Hi folks,

I think this was discussed yesterday in the meeting, and a bit on
slack, but I haven't seen anything much written down, so I'm starting
a thread here.

The crux of the problem is that the protobuf text format currently
used for metadata cannot have new fields added without breaking client
compatability, as the text parser can't be configured to ignore
unrecognised values (and google aren't going to fix this).

Protobuf binary format does support new fields though. So if a field
is added, a client that knows nothing of it can read it back without
issue.

I propose we approach this the following way:
- We already have a version in /ledgers/LAYOUT. In a current cluster,
this contains
```
2
org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory:1
```
- We define a new LedgerMetadata protobuf. This is a chance to clean
up mistakes we've made previously.
- When writing a metadata, check what is in /ledgers/LAYOUT. If it is
as above, write using the current text protobuf. If is bumped, use the
new binary format.
- When reading metadata, first try to parse binary, and fall back to
text if that fails. (we could also add a layout check to short
circuit)

When upgrading a cluster, the layout will be as above, it will
continue to only use text format until there is some admin
intervention. When the admin is satisfied that all clients are on a
new enough version, they call a script which bumps the version. From
this point clients will write the binary version.

New clusters go straight to binary. We will also need a script to dump
the metadata from a znode. One wrinkle, which is another reason to
create a new LedgerMetadata protobuf, is that when you are writing in
text format, there's no way to filter the fields. So even if we write
in text format, if we add new fields we are breaking old clients.
Another approach would be to ensure the protobuf only contains the
fields that are available now, but this seems messy to me.

Anyhow, this is to be a jumping off point for discussion. Comments welcome,

Cheers,
Ivan


Re: Official Docker Images

2018-05-29 Thread Ivan Kelly
+1 from me. Changing the tag was really bad practice. Good to see this changing.

On Tue, May 29, 2018 at 6:59 AM, Enrico Olivelli  wrote:
> Il mar 29 mag 2018, 02:50 Sijie Guo  ha scritto:
>
>> Since I am going to cut a new release 4.7.1, I would like to change the
>> release procedure for docker images, we don't need to retag release tags
>> for updating docker files.
>>
>> Please take a look at this BP -
>> https://github.com/apache/bookkeeper/pull/1450
>
>
> Already gave my +1
> Very good thank you
>
> Enrico
>
>>
>>
>>
>>
>>
>> - Sijie
>>
>> On Wed, Apr 18, 2018 at 12:21 PM, Jia Zhai  wrote:
>>
>> > +1
>> >
>> > On Tue, Apr 17, 2018 at 2:17 PM, Enrico Olivelli 
>> > wrote:
>> >
>> > > Il mar 17 apr 2018, 20:57 Sijie Guo  ha scritto:
>> > >
>> > > > Hi all,
>> > > >
>> > > > Current docker images are auto-built by apache docker account. It
>> > > becomes a
>> > > > bit problematic in the release process. We might consider moving the
>> > > docker
>> > > > build process out of apache main repo.
>> > > >
>> > > > So I am restarting the discussion about "official docker image",
>> > > following
>> > > > what flink is doing. flink maintains a separate repo:
>> > > > https://github.com/docker-flink and update docker "official-images"
>> > > repo.
>> > > >
>> > > > We can do the similar thing:
>> > > >
>> > > > - create an `apachebookkeeper` org and a `docker` repo under it
>> > > > - update docker "official-images"
>> > > >
>> > > > The docker repo will be managed under bookkeeper pmc. So docker files
>> > > > update will only be done when a bookkeeper release is completed,
>> which
>> > > > would make the release sequence clearer and also we are able to
>> publish
>> > > > "docker-official" images.
>> > > >
>> > > > Thoughts?
>> > > >
>> > >
>> > > Totally +1
>> > > I really don't like that re-tagging procedure.
>> > >
>> > > Enrico
>> > >
>> > >
>> > >
>> > > > [1] https://docs.docker.com/docker-hub/official_repos/
>> > > > [2] https://github.com/docker-library/docs
>> > > > [3] https://flink.apache.org/news/2017/05/16/official-docker-
>> > image.html
>> > > >
>> > > > - Sijie
>> > > >
>> > > --
>> > >
>> > >
>> > > -- Enrico Olivelli
>> > >
>> >
>>
> --
>
>
> -- Enrico Olivelli


Re: [ANNOUNCE] Apache BookKeeper 4.6.2 released

2018-04-10 Thread Ivan Kelly
Great work getting this out Enrico!

Cheers,
Ivan

On Tue, Apr 10, 2018 at 9:28 AM, Enrico Olivelli  wrote:
> The Apache BookKeeper team is proud to announce Apache BookKeeper version
> 4.6.2.
>
> Apache BookKeeper is a scalable, fault-tolerant, and low-latency storage
> service optimized for
> real-time workloads. It has been used for a fundamental service to build
> reliable services.
> It is also the log segment store for Apache DistributedLog and the message
> store for Apache Pulsar.
>
> This is the 9-th release of Apache BookKeeper.
>
> This is a bugfix release, it fixes bugs around Java 9/10 support and it
> brings performance enhancements on Prometheus stats provider and on client
> side memory usage.
>
> For BookKeeper release details and downloads, visit:
>
> https://bookkeeper.apache.org/releases/
>
> BookKeeper 4.6.2 Release Notes are at:
>
> https://bookkeeper.apache.org/docs/4.6.2/overview/releaseNotes/
>
> We would like to thank the contributors that made the release possible.
>
> Regards,
>
> The BookKeeper Team


Re: [VOTE] Apache BookKeeper Release 4.6.2, release candidate #2

2018-04-03 Thread Ivan Kelly
+1 lgtm

RAT, FINDBUGS & TESTS:

Runs cleanly. I had -Dsurefire.rerunFailingTestsCount=2 set, but I
don't think anything even flaked.

Minor issue: Vertx http tests expect 8080 to be free

SHA1 & SIGs:

Good.

LICENSE & NOTICE:

Nothing changed since 4.6.1, so looks good.

Minor: Copyright in notice should include 2018

-Ivan

On Tue, Apr 3, 2018 at 1:57 PM, Enrico Olivelli  wrote:
> Hi everyone,
> Please review and vote on the release candidate #2 for the version 4.6.2,
> as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
>
> The complete staging area is available for your review, which includes:
> * Release notes [1]
> * The official Apache source and binary distributions to be deployed to
> dist.apache.org [2]
> * All artifacts to be deployed to the Maven Central Repository [3]
> * Source code tag "release-4.6.2" [4] with git sha
> f142315ec18deec84cd2acf28aa4a8a932b24baf
>
> BookKeeper's KEYS file contains PGP keys we used to sign this release:
> https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
>
> Please download these packages and review this release candidate:
>
> - Review release notes
> - Download the source package (verify shasum, and asc) and follow the
> instructions to build and run the bookkeeper service.
> - Download the binary package (verify shasum, and asc) and follow the
> instructions to run the bookkeeper service.
> - Review maven repo, release tag, licenses, and any other things you think
> it is important to a release.
>
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
>
> Thanks,
> Enrico Olivelli
>
> [1] https://github.com/apache/bookkeeper/pull/1297/
> [2] https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper-4.6.2-rc2/
> [3]
> https://repository.apache.org/content/repositories/orgapachebookkeeper-1026/
> [4] https://github.com/apache/bookkeeper/commits/release-4.6.2


Re: Old DistributedLog 0.5.0 RC1 on dist.apache.org

2018-04-03 Thread Ivan Kelly
Kill it. If 0.5.0 is out, then it serves no purpose.

-Ivan


On Tue, Apr 3, 2018 at 1:36 PM, Enrico Olivelli  wrote:
> Hi,
> there is this old directory
> https://dist.apache.org/repos/dist/dev/bookkeeper/distributedlog/0.5.0-rc1/
>
> can I drop it ?
> I think Jia left it during the release process of 0.5.0
>
> Enrico


Re: [VOTE] Apache BookKeeper Release 4.6.2, release candidate #1

2018-04-03 Thread Ivan Kelly
On Tue, Apr 3, 2018 at 12:48 PM, Enrico Olivelli  wrote:
> We found an issue and this is the fix
> https://github.com/apache/bookkeeper/pull/1312
>
> Will send a new RC as soon as the patch is merged to branch-4.6

Taking a look.

-Ivan


Re: [VOTE] Apache BookKeeper Release 4.6.2, release candidate #1

2018-04-03 Thread Ivan Kelly
> I cannot check on the 4.6.0 one because we are only keeping the latest
> version on dist.apache.org

archive.apache.org has older releases.

-Ivan


Re: [VOTE] Apache BookKeeper Release 4.6.2, release candidate #1

2018-04-03 Thread Ivan Kelly
Hey Enrico,

Thanks for putting the release together. I'm afraid there's an issue
with the source package though.

~/blah/4.6.2-rc1 $ ls -l bookkeeper-4.6.2/bookkeeper-server/bin
total 20
-rw-r--r-- 1 ivan ivan 7364 Mar 28 09:24 bookkeeper
-rw-r--r-- 1 ivan ivan 2869 Mar 28 09:24 bookkeeper-cluster.sh
-rw-r--r-- 1 ivan ivan 4357 Mar 28 09:24 bookkeeper-daemon.sh

This also makes some tests fail (the rack awareness with script one).

So, a -1 from me, for the package that's up there.

Otherwise everything else was looking good. I may not even need to cut
a new RC though, since this looks like a package generation issue, not
something that needs to be in the branch. Did you generate on a fat32
filesystem or something?

Cheers,
Ivan

On Tue, Apr 3, 2018 at 9:26 AM, Enrico Olivelli  wrote:
> Hi everyone,
> Please review and vote on the release candidate #1 for the version 4.6.2,
> as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
>
> The complete staging area is available for your review, which includes:
> * Release notes [1]
> * The official Apache source and binary distributions to be deployed to
> dist.apache.org [2]
> * All artifacts to be deployed to the Maven Central Repository [3]
> * Source code tag "release-4.6.2" [4] with git sha
> 70c613f1700d4c19e36c10b5b51e841b138569fb
>
> BookKeeper's KEYS file contains PGP keys we used to sign this release:
> https://dist.apache.org/repos/dist/release/bookkeeper/KEYS
>
> Please download these packages and review this release candidate:
>
> - Review release notes
> - Download the source package (verify shasum, and asc) and follow the
> instructions to build and run the bookkeeper service.
> - Download the binary package (verify shasum, and asc) and follow the
> instructions to run the bookkeeper service.
> - Review maven repo, release tag, licenses, and any other things you think
> it is important to a release.
>
> The vote will be open for at least 72 hours. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
>
> Thanks,
> Enrico Olivelli
>
> [1] https://github.com/apache/bookkeeper/pull/1297/
> [2]
> https://dist.apache.org/repos/dist/dev/bookkeeper/bookkeeper/bookkeeper-4.6.2-rc1/
> [3]
> https://repository.apache.org/content/repositories/orgapachebookkeeper-1025/
> [4] https://github.com/apache/bookkeeper/commits/release-4.6.2


Re: BK metrics

2018-03-21 Thread Ivan Kelly
> @Ivan, for some reasons I did not receive your reply but found it in the 
> email archives.

Are you subscribed to the list? I did see one mail from you show up in
moderation.

> At 80K request/sec throttling for record size of 1K, I am getting below 
> throughput. The 99th percentile of `bookkeeper_server_ADD_ENTRY_REQUEST` and 
> `bookkeeper_server_ADD_ENTRY` are around 350 ms. I am starting to see the lag 
> when I increase the ingestion rate limit beyond 90 K/sec limit.

So this suggests to me that the metrics are reporting correctly.

> The disks were under 60% utilization (not saturated).

60% of bandwidth or iops? only one of the two needs to be saturated.
And which disk, journal or ledgers?

> Are there any benchmark results of BookKeeper that can be shared?

I don't have any to hand, but maybe someone else on the list does.

Regards,
Ivan


Re: BK metrics

2018-03-20 Thread Ivan Kelly
> 2) If it's in milliseconds, are these numbers in expected range (see
> attached image). To me 2.5 seconds  (2.5K ms) latency for add entry request
> is very high.
2.5 seconds is very high, but your write rate is also high. 100,000 *
1KB is 100MB/s. SSD should be able to take it from the journal side,
but it depends on the hardware.

Have you tried reducing the write rate to see how the latency changes?
What is the client seeing for latency? I assume the client and all
servers have  10GigE nics?

Your images didn't attach correctly. Maybe they're too big to post
directly to the list. There is a size limit, but I don't know what it
is.

-Ivan


Re: [DISCUSS] Inconsistency in Handle based APIs - Specifically "close"

2018-03-20 Thread Ivan Kelly
On Mon, Mar 19, 2018 at 6:37 PM, Sijie Guo  wrote:
> It is not a blocker for me.
>
> But if we want consistency, either applying pattern "asyncXYZ()" or
> "xyzAsync()" for async operations works for me.
xyzAsync is better than asyncXyz, as it will put the async and sync
versions together in the javadoc.

So, there's 3 options here.

1. Remove Closeable
2. Some kind of split of sealing and close
3. Create sync and async versions of all with Async suffix.

I think 3 is the most palatable. If there's no objections I'll push a
patch later today.

-Ivan


Re: [DISCUSS] Inconsistency in Handle based APIs - Specifically "close"

2018-03-19 Thread Ivan Kelly
> Is implementing Closable a "valueable" feature for us in the new API ?  (I
> think the answer is 'yes')

I'm not so sure how useful Closeable is here. It is handy in tests,
but in production code you are never going to use the
try-with-resources pattern, as you'll be using async calls for
everything else. I think people not waiting on a returned
CompletableFuture is a bigger issue, but maybe even that isn't
important (see below).

> There was a discussion about introducing some CompletableFuture seal()
> method, which would be more like current close().

Yes, there was a BP which just ended going around in circles and I got
frustrated and closed it. Part of the problem was there were many
things being discusses at the same time, so now I just want to
concentrate on the close inconsistency.

> With this approach we should document very well that a seal() must be
> called and about the risks of not calling that seal()

What are those risks? If I never called close() or seal() on my
WriteHandle, what is the worst thing that could happen? I don't think
much bad could happen at all, because not calling close or seal is the
same as crashing before you do. So not calling it means that the next
person to open the ledger has to deal with it, which is a latency hit
for them. Even in the case where you are rolling your ledger, and then
continue writing to a new one, I don't think there's a problem. If you
don't close, then any entries that have been successfully written will
continue to be successfully written, so whether you record this in the
metadata isn't important. And you don't need to fence, because in this
case you are the writer.

Anyhow, I'm not arguing for a particular solution here, just
highlighting that the operation of "close" isn't as vital as it has
always appeared to be.

Cheers,
Ivan


[DISCUSS] Inconsistency in Handle based APIs - Specifically "close"

2018-03-19 Thread Ivan Kelly
Hi folks,

I'm currently changing some parts of pulsar to use the new APIs and
the inconsistency in the close api has raised its head again, so I'm
restarting this discussion.

Handle has the following methods:
async: asyncClose
sync: close, getId, getLedgerMetadata

ReadHandle has the following methods:
async: read, readUnconfirmed, readLastAddConfirmed,
tryReadLastAddConfirmed, readLastAddConfirmedAndEntry
sync: isClosed, getLength, getLastAddConfirmed

WriteHandle has the following methods:
async: append
sync: getLastAddPushed

Close is inconsistent with the rest of the methods for a number of reasons.
1. No other async method uses the async* pattern.
2. All other sync methods are querying local data and are sideeffect
free. Close can trigger I/O.
3. Each other method has one way be being called, close has two.

I'm not going to suggest a solution to this right now, but any
solution which gets rid of this inconsistency would be acceptable.

New APIs shouldn't have inconsistencies like this from the outset, and
this is a blocker for me for moving the API away from the Unstable
annotations.

What are your thoughts?

Cheers,
Ivan


Reporting CI failures to github issues

2018-03-19 Thread Ivan Kelly
Hi folks,

When you report a CI failure to github, can you click on "Keep this
build forever" in jenkins itself, so that the results will still be
around when we eventually look at it.
Unfortunately, you need to log in for this.

Cheers,
Ivan


Re: Drop md5 from release procedure

2018-03-19 Thread Ivan Kelly
> Can you forward the discussion thread if there is one?

It was a mail to the private list, subject "checksum file Release
Distribution Policy"

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-16 Thread Ivan Kelly
> With "paranoid" log in Netty I found this that is very interesting, but it
> happens even on Java 8.
I don't think leaks are the problem here though. This seems to be more
like a doublefree issue.

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-15 Thread Ivan Kelly
> What is the difference in Channel#write/ByteBuf pooling.in Java 9 ?
Sounds like it could be an issue in netty itself. Java 9 removed a
bunch of stuff around Unsafe, which I'm pretty sure netty was using
for ByteBuf. Have you tried setting the pool debugging to paranoid?

-Dio.netty.leakDetection.level=paranoid


i tried running my attempted repro in java9, but no cigar.


-Ivan


Re: Help with bad errors on 4.6.1

2018-03-14 Thread Ivan Kelly
>> > @Ivan
>> > I wonder if some tests on Jepsen with bookie restarts may find this kind
>> of
>> > issues, given that it is not a network/SO problem
>> If jepsen can catch then normal integration test can.

I attempted a repro for this using the integration test stuff.
Running for 2-3 hours in a loop, no bug hit. Perhaps I'm not doing
exactly what you are doing.

https://github.com/ivankelly/bookkeeper/blob/enrico-bug/tests/integration/enrico-bug/src/test/java/org/apache/bookkeeper/tests/integration/TestEnricoBug.java

-Ivan


Re: Jenkins broken this morning

2018-03-14 Thread Ivan Kelly
Broken again, infra are working on it.

On Wed, Mar 14, 2018 at 1:10 PM, Ivan Kelly <iv...@apache.org> wrote:
> Hi folks,
>
> Jenkins went nuts this morning, so infra rebooted it. It means that
> some jobs that may have been pending never happened, so rekick the
> testing on your patches if needed (retest this please).
>
> Cheers
> Ivan


Jenkins broken this morning

2018-03-14 Thread Ivan Kelly
Hi folks,

Jenkins went nuts this morning, so infra rebooted it. It means that
some jobs that may have been pending never happened, so rekick the
testing on your patches if needed (retest this please).

Cheers
Ivan


Re: Help with bad errors on 4.6.1

2018-03-13 Thread Ivan Kelly
> @Ivan
> I wonder if some tests on Jepsen with bookie restarts may find this kind of
> issues, given that it is not a network/SO problem
If jepsen can catch then normal integration test can. The readers in
question, are they tailing with long poll, or just calling
readLastAddConfirmed in a loop? What is the configuration in terms of
ensemble/write/ack?

I can try to put together a repro too, using the integ test framework.

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-12 Thread Ivan Kelly
> It is interesting that the problems is on 'readers' and it seems that the
> PCBC seems corrupted and even writes (if the broker is promoted to
> 'leader') are able to go on after the reads broke the client.
Are writes coming from the same clients? Or clients in the same process?

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-12 Thread Ivan Kelly
> - when I "restart" bookies I issue a kill -9 (I think this could be the
> reason why I can't reproduce the issue on testcases)
With a clean shutdown of bookies we close the channels, and it should
do the tcp shutdown handshake. -9 will kill the process before it gets
to do any of that, but the kernel will kill the socket. Pretty sure it
doesn't do the fin-ack handshake. I suspect that netty should be
throwing up a different exception in this case.

You can't do a kill -9 with unit tests, but with integration tests
it's possible.

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-12 Thread Ivan Kelly
ocol$Response.parseFrom(BookkeeperProtocol.java:8454)
> at
> org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDecoderV3.decode(BookieProtoEncoding.java:329)
> at
> org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseDecoder.decode(BookieProtoEncoding.java:470)
> at
> io.netty.handler.codec.MessageToMessageDecoder.channelRead(MessageToMessageDecoder.java:88)
> ... 19 more
>
>
> 2018-03-09 22:37 GMT+01:00 Sijie Guo <guosi...@gmail.com>:
>
>> Enrico,
>>
>> I would suggest you applied my fixes and then debug from there. In this
>> way, you will have a better sense where the first corruption is from.
>>
>> Sijie
>>
>> On Fri, Mar 9, 2018 at 11:48 AM Enrico Olivelli <eolive...@gmail.com>
>> wrote:
>>
>> > Il ven 9 mar 2018, 19:30 Enrico Olivelli <eolive...@gmail.com> ha
>> scritto:
>> >
>> > > Thank you Ivan!
>> > > I hope I did not mess up the dump and added ZK ports.  We are not using
>> > > standard ports and in that 3  machines there is also the 3 nodes zk
>> > > ensemble which is supporting BK and all the other parts of the
>> > application
>> > >
>> > > So one explanation would be that something is connecting to the bookie
>> > and
>> > > this makes the bookie switch in a corrupted state by double releasing a
>> > > bytebuf?
>> > >
>> >
>> > I did some experiments and it is easy to reproduce the bookie side error
>> > and the double release with a forged sequence of bytes (just using nc
>> from
>> > the shell)
>> > But this seems not enough to break the bookie.
>> > I guess there is some corruption on client side and the error on the
>> bookie
>> > is only and effect, as Ivan is saying.
>> > My colleagues left the system running  with a deep level of debug during
>> > next weekend, hopefully we will get some other stacktrace
>> >
>> > Enrico
>> >
>> >
>> > > Enrico
>> > >
>> > >
>> > > Il ven 9 mar 2018, 18:23 Ivan Kelly <iv...@apache.org> ha scritto:
>> > >
>> > >> I need to sign off for the day. I've done some analysis of a tcpdump
>> > >> enrico sent to me out of band (may contain sensitive info so best not
>> > >> to post on public forum).
>> > >>
>> > >> I've attached a dump of just first bit of the header. Format is
>> > >>  ()
>> > >> 
>> > >>
>> > >> There are definitely corrupt packets coming from somewhere. Search for
>> > >> lines with CORRUPT.
>> > >>
>> > >> 
>> > >> 0247 -  req (049546) - 00:00:00:08:ff:ff:ff:fe:00:00:00:0bCORRUPT
>> > >> 
>> > >>
>> > >> It's not clear whether these are originating at a valid client or not.
>> > >> These trigger corrupt responses from the server, which I guess is the
>> > >> double free manifesting itself. Strangely the
>> > >> corrupt message seems to have a lot of data in common with what seems
>> > >> like an ok message (it's clearer on fixed width font).
>> > >>
>> > >> 
>> > >> 0248 -  resp(049720) -
>> > >>
>> > >>
>> > 00:00:00:54:00:03:00:89:00:00:02:86:00:07:e2:b1:00:00:00:00:
>> 00:00:02:86:00:05:e9:76:00:00
>> > >> 0249 -  resp(049546) -
>> > >> 00:00:00:10:ff:ff:ff:fe:00:00:02:86:00:07:e2:b1:00:00:00:00
>> CORRUPT
>> > >> 
>> > >>
>> > >> There's also some other weird traffic. Correct BK protobuf traffic
>> > >> should be <4 bytes len>:00:03:
>> > >> There seems to be other traffic which is being accepted at the same
>> > >> port, but looks like ZK traffic.
>> > >>
>> > >> Anyhow, I'll dig more on monday.
>> > >>
>> > >> -Ivan
>> > >>
>> > >>
>> > >> On Fri, Mar 9, 2018 at 3:27 PM, Ivan Kelly <iv...@apache.org> wrote:
>> > >> > On Fri, Mar 9, 2018 at 3:20 PM, Enrico Olivelli <
>> eolive...@gmail.com>
>> > >> wrote:
>> > >> >> Bookies
>> > >> >> 10.168.10.117:1822 -> bad bookie with 4.1.21
>> > >> >> 10.168.10.116:1822 -> bookie with 4.1.12
>> > >> >> 10.168.10.118:1281 -> bookie with 4.1.12
>> > >> >>
>> > >> >> 10.168.10.117 client machine on which I have 4.1.21 client
>> (different
>> > >> >> process than the bookie one)
>> > >> > Oh. This dump won't have the stream we need then, as that will be on
>> > >> > loopback. Try adding "-i any" to the tcpdump. Sorry, I didn't
>> realize
>> > >> > your clients and servers are colocated.
>> > >> >
>> > >> > -Ivan
>> > >>
>> > > --
>> > >
>> > >
>> > > -- Enrico Olivelli
>> > >
>> > --
>> >
>> >
>> > -- Enrico Olivelli
>> >
>>


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
On Fri, Mar 9, 2018 at 3:20 PM, Enrico Olivelli  wrote:
> Bookies
> 10.168.10.117:1822 -> bad bookie with 4.1.21
> 10.168.10.116:1822 -> bookie with 4.1.12
> 10.168.10.118:1281 -> bookie with 4.1.12
>
> 10.168.10.117 client machine on which I have 4.1.21 client (different
> process than the bookie one)
Oh. This dump won't have the stream we need then, as that will be on
loopback. Try adding "-i any" to the tcpdump. Sorry, I didn't realize
your clients and servers are colocated.

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
Also, do you have the logs of the error occurring on the server side?

-Ivan

On Fri, Mar 9, 2018 at 3:16 PM, Ivan Kelly <iv...@apache.org> wrote:
> On Fri, Mar 9, 2018 at 3:13 PM, Enrico Olivelli <eolive...@gmail.com> wrote:
>> New dump,
>> sequence (simpler)
>>
>> 1) system is running, reader is reading without errors with netty 4.1.21
>> 2) 3 bookies, one is with 4.1.21 and the other ones with 4.1.12
>> 3) kill one bookie with 4.1.12, the reader starts reading from the bookie
>> with 4.1.21
>> 4) client messes up, unrecoverably
>
> What port is bookie running on? And the ip of the client?
>
> -Ivan


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
On Fri, Mar 9, 2018 at 3:13 PM, Enrico Olivelli  wrote:
> New dump,
> sequence (simpler)
>
> 1) system is running, reader is reading without errors with netty 4.1.21
> 2) 3 bookies, one is with 4.1.21 and the other ones with 4.1.12
> 3) kill one bookie with 4.1.12, the reader starts reading from the bookie
> with 4.1.21
> 4) client messes up, unrecoverably

What port is bookie running on? And the ip of the client?

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
I've asked enrico to run again, as this dump doesn't span the time
when the issue started occurring.
What I'm looking for is to be able to inspect the first packet which
triggers the version downgrade of the decoders.


On Fri, Mar 9, 2018 at 3:04 PM, Enrico Olivelli  wrote:
> This is the dump
>
> in order to reproduce the issue I did these steps:
> - all bookies with 4.1.12
> - start reader with netty 4.1.21 -> still no issue
> - start on bookie with 4.1.21 -> start issue
> - bookie again with 4.1.12 -> client messed up, unrecoverable
>
> a strange fact is that I cannot find IllegalReferenceCount errors on bookies
>
> I will try different combinations in order to create a simpler reproducer
>
> that you for you help


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
> Any suggestion on the tcpdump config ? (command line example)

sudo tcpdump -s 200 -w blah.pcap 'tcp port 3181'

Where are you going to change the netty? client or server or both?

-Ivan


Re: Help with bad errors on 4.6.1

2018-03-09 Thread Ivan Kelly
Great analysis Sijie.

Enrico, are these high traffic machines? Would it be feasible to put
tcpdump running? You could even truncate each message to 100 bytes or
so, to avoid storing payloads. It'd be very useful to see what the
corrupt traffic actually looks like.

-Ivan

On Fri, Mar 9, 2018 at 10:43 AM, Enrico Olivelli  wrote:
> Reverted to Netty 4.1.12. System is "more" stable but after "some" restart
> we still have errors on client side on tailing readers, rebooting the JMV
> "resolved" temporary the problem.
>
> I have no more errors on the Bookie side
>
> My  idea:
> - client is reading from 2 bookies, there is some bug in this area
> - once the reader client messes up Netty stops working properly and
> corrupts ByteBufs
> - once the reader JMV is promoted to writer (becomes leader and starts
> writing) its internal BookKeeper client is still messed up and writes are
> not able to complete
>
>
> Enrico
>
>
>
>
> 2018-03-09 9:55 GMT+01:00 Enrico Olivelli :
>
>>
>>
>> 2018-03-09 8:59 GMT+01:00 Sijie Guo :
>>
>>> Sent out a PR for the issues that I observed:
>>>
>>> https://github.com/apache/bookkeeper/pull/1240
>>>
>>
>>
>> Other findings:
>> - my problem is not related to jdk9, it happens with jdk8 too
>> - the "tailing reader" is able to make progress and follow the WAL, so not
>> all the reads fail
>> - the "writer" is able to make progress and write to the WAL, so not all
>> the write fail
>>
>> I have run BK 4.6.1 tests with Netty 4.1.21Final but there is no issue
>> (quite the OS as the machines with the error, I am on Fedora, machines are
>> on CentOS)
>>
>> Enrico
>>
>>
>>>
>>>
>>> On Thu, Mar 8, 2018 at 10:47 PM, Sijie Guo  wrote:
>>>
 So the problem here is:

 - a corrupted request failed the V3 request decoder, so bookie switched
 to use v2 request decoder. Once the switch happen, the bookie will always
 use v2 request decoder decoding v3 request. then all your v3 requests will
 be failing with unknown op and trigger the bytebuf double releasing issue.

 - a corrupted response failed the V3 response decoder, so client
 switched to use v3 response decoder. Once the switch happen, the client
 will always use v2 request decoder decoding v3 response. so all the
 responses will be failing with " Received unknown response : op code"

 Although I don't know how the first request/response happened (it can be
 any issue, even network corruption), the problem is once this happen,
 either client/bookie will be forcing to use v2 request/response decoder and
 never change. so the problem will remain until you restarted. And this is
 the behavior that Enrico is seeing.

 There are a couple of issues to address here:

 1) we need to add a flag to disable falling back to use v2
 request/response coder and make sure it always use v3 protocol. In this
 way, we will guarantee the problem not persist in the channel level.
 2) we need to throw exception on unknown op code at request decode :
 https://github.com/apache/bookkeeper/blob/master/bookkeepe
 r-server/src/main/java/org/apache/bookkeeper/proto/BookiePro
 toEncoding.java#L195 . As what we did at response decoder :
 https://github.com/apache/bookkeeper/blob/master/bookkeepe
 r-server/src/main/java/org/apache/bookkeeper/proto/BookiePro
 toEncoding.java#L304 in https://github.com/apache/bookkeeper/issues/198


 Details are listed as below:

 --

 Details:

 - The client side stacktrace clearly showed that it is using v2 decoder
 on decoding responses. That means client failed to parse v3 response and it
 falls back to use v2 decoder on decoding responses. Because it is a
 "corrupted" v3 response, so v2 decoder can't
 find a right op code.  Then it throws illegal state exception.


 *Caused by: java.lang.IllegalStateException: Received unknown response :
 op code = 6at
 org.apache.bookkeeper.proto.BookieProtoEncoding$ResponseEnDeCoderPreV3.decode(BookieProtoEncoding.java:294)*

 - For the stacktrace at bookie side:

 1. It is clear that BookieRequestHandler#L77 is called. That means the
 message is neither v2 request nor v3 request. It is a ByteBuf.

 https://github.com/apache/bookkeeper/blob/master/bookkeeper-
 server/src/main/java/org/apache/bookkeeper/proto/BookieReque
 stHandler.java#L77

 2. V3 request decoder is using protobuf to decode bytes array. If it is
 not a valid v3 request, it will always throw exceptions. so the code
 mentioned above will never be triggered

 https://github.com/apache/bookkeeper/blob/master/bookkeeper-
 server/src/main/java/org/apache/bookkeeper/proto/BookieProto
 Encoding.java#L344

 3. The only case that BookieRequestHandler#L77 can happen is v2 request
 decoder 

Re: Replication Worker and targetBookie.

2018-03-09 Thread Ivan Kelly
> The "predicate" approach is problematic, it can potentially cause some
> ledgers never being replicated. Ideally, this is something should be done
> by auditor, because auditor
> knows the ledgers, the alive bookies and the network topology, auditor
> should be able to compute a replication plan and assign corresponding
> ledgers to bookies. This will ensure:
>
> - optimize the placement
> - ensure no ledgers will be missed

+1 on doing this in the auditor. This getting very much into
distributed scheduler territory.

-Ivan


  1   2   3   >