I was thinking that we might need something like multi-operation
<https://issues.apache.org/jira/browse/ZOOKEEPER-965> record in zookeeper
to atomically create topic and partition records when this multi record is
committed.  This way metadata will have both the TopicRecord and
PartitionRecord together always, and in no situation we can have
TopicRecord without PartitionRecord. Not sure if there are other situations
where multi-operation is needed.
<https://issues.apache.org/jira/browse/ZOOKEEPER-965>

Thanks,
Unmesh

On Fri, Jul 10, 2020 at 11:32 AM Colin McCabe <cmcc...@apache.org> wrote:

> Hi Unmesh,
>
> Yes, once the last stable offset advanced, we would consider the topic
> creation to be done, and then we could return success to the client.
>
> best,
> Colin
>
> On Thu, Jul 9, 2020, at 19:44, Unmesh Joshi wrote:
> > It still needs HighWaterMark / LastStableOffset to be advanced by two
> > records? Something like following?
> >
> >
> >                        |                |
> > <------------------    |----------------|   HighWaterMark
> >    Response            |PartitionRecord |
> >                        |                |
> >                        -----------------|
> >                        | TopicRecord    |                          -
> >                        |                |
> > ------------------->   ------------------   Previous HighWaterMark
> >    CreateTopic         |                |
> >                        |                |
> >                        |                |
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> >
> > On Fri, Jul 10, 2020 at 1:30 AM Colin McCabe <cmcc...@apache.org> wrote:
> >
> > > On Thu, Jul 9, 2020, at 04:37, Unmesh Joshi wrote:
> > > > I see that, when a new topic is created, two metadata records, a
> > > > TopicRecord (just the name and id of the topic) and a PartitionRecord
> > > (more
> > > > like LeaderAndIsr, with leader id and replica ids for the partition)
> are
> > > > created.
> > > > While creating the topic, log entries for both the records need to be
> > > > committed in RAFT core. Will it need something like a
> > > MultiOperationRecord
> > > > in zookeeper. Then, we can have a single log entry with both the
> records,
> > > > and  the create topic request can be fulfilled atomically when both
> the
> > > > records are committed?
> > > >
> > >
> > > Hi Unmesh,
> > >
> > > Since the active controller is the only node writing to the log, there
> is
> > > no need for any kind of synchronization or access control at the log
> level.
> > >
> > > best,
> > > Colin
> > >
> > > >
> > > > Thanks,
> > > > Unmesh
> > > >
> > > > On Wed, Jul 8, 2020 at 6:57 AM Ron Dagostino <rndg...@gmail.com>
> wrote:
> > > >
> > > > > HI Colin.  Thanks for the KIP.  Here is some feedback and various
> > > > > questions.
> > > > >
> > > > > "*Controller processes will listen on a separate port from brokers.
> > > This
> > > > > will be true even when the broker and controller are co-located in
> the
> > > same
> > > > > JVM*". I assume it is possible that the port numbers could be the
> same
> > > when
> > > > > using separate JVMs (i.e. broker uses port 9192 and controller also
> > > uses
> > > > > port 9192).  I think it would be clearer to state this along these
> > > > > lines: "Controller
> > > > > nodes will listen on a port, and the controller port must differ
> from
> > > any
> > > > > port that a broker in the same JVM is listening on.  In other
> words, a
> > > > > controller and a broker node, when in the same JVM, do not share
> ports"
> > > > >
> > > > > I think the sentence "*In the realm of ACLs, this translates to
> > > controllers
> > > > > requiring CLUSTERACTION on CLUSTER for all operations*" is
> confusing.
> > > It
> > > > > feels to me that you can just delete it.  Am I missing something
> here?
> > > > >
> > > > > The KIP states "*The metadata will be stored in memory on all the
> > > active
> > > > > controllers.*"  Can there be multiple active controllers?  Should
> it
> > > > > instead read "The metadata will be stored in memory on all
> potential
> > > > > controllers." (or something like that)?
> > > > >
> > > > > KIP-595 states "*we have assumed the name __cluster_metadata for
> this
> > > > > topic, but this is not a formal part of this proposal*".  This
> KIP-631
> > > > > states "*Metadata changes need to be persisted to the __metadata
> log
> > > before
> > > > > we propagate them to the other nodes in the cluster.  This means
> > > waiting
> > > > > for the metadata log's last stable offset to advance to the offset
> of
> > > the
> > > > > change.*"  Are we here formally defining "__metadata" as the topic
> > > name,
> > > > > and should these sentences refer to "__metadata topic" rather than
> > > > > "__metadata log"?  What are the "other nodes in the cluster" that
> are
> > > > > referred to?  These are not controller nodes but brokers, right?
> If
> > > so,
> > > > > then should we say "before we propagate them to the brokers"?
> > > Technically
> > > > > we have a controller cluster and a broker cluster -- two separate
> > > clusters,
> > > > > correct?  (Even though we could potentially share JVMs and
> therefore
> > > > > require no additional processes.). If the statement is referring to
> > > nodes
> > > > > in both clusters then maybe we should state "before we propagate
> them
> > > to
> > > > > the other nodes in the controller cluster or to brokers."
> > > > >
> > > > > "*The controller may have several of these uncommitted changes in
> > > flight at
> > > > > any given time.  In essence, the controller's in-memory state is
> > > always a
> > > > > little bit in the future compared to the current state.  This
> allows
> > > the
> > > > > controller to continue doing things while it waits for the previous
> > > changes
> > > > > to be committed to the Raft log.*"  Should the three references
> above
> > > be to
> > > > > the active controller rather than just the controller?
> > > > >
> > > > > "*Therefore, the controller must not make this future state
> "visible"
> > > to
> > > > > the rest of the cluster until it has been made persistent – that
> is,
> > > until
> > > > > it becomes current state*". Again I wonder if this should refer to
> > > "active"
> > > > > controller, and indicate "anyone else" as opposed to "the rest of
> the
> > > > > cluster" since we are talking about 2 clusters here?
> > > > >
> > > > > "*When the active controller decides that it itself should create a
> > > > > snapshot, it will first try to give up the leadership of the Raft
> > > quorum.*"
> > > > > Why?  Is it necessary to state this?  It seems like it might be an
> > > > > implementation detail rather than a necessary
> constraint/requirement
> > > that
> > > > > we declare publicly and would have to abide by.
> > > > >
> > > > > "*It will reject brokers whose metadata is too stale*". Why?  An
> > > example
> > > > > might be helpful here.
> > > > >
> > > > > "*it may lose subsequent conflicts if its broker epoch is stale*"
> This
> > > is
> > > > > the first time a "broker epoch" is mentioned.  I am assuming it is
> the
> > > > > controller epoch communicated to it (if any).  It would be good to
> > > > > introduce it/explicitly state what it is before referring to it.
> > > > >
> > > > > Ron
> > > > >
> > > > > On Tue, Jul 7, 2020 at 6:48 PM Colin McCabe <cmcc...@apache.org>
> > > wrote:
> > > > >
> > > > > > Hi all,
> > > > > >
> > > > > > I posted a KIP about how the quorum-based controller envisioned
> in
> > > > > KIP-500
> > > > > > will work.  Please take a look here:
> > > > > > https://cwiki.apache.org/confluence/x/4RV4CQ
> > > > > >
> > > > > > best,
> > > > > > Colin
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to