Thank you for the really well thought out KIP. This is a very neat approach
to solve a rather complex problem.

I haven't yet digested the doc. But I have a few comments and questions.

11 a). It feels like memory usage in the consumer is unbounded. If an
application crashed mid-transaction, lots of messages from other
transactions could be written before the transaction timeout eventually
aborted the first transaction. Any consumer reading the log would need to
wait and buffer all the messages before actually consuming any. You could
end up with a consumer that can never consume from the log even if
restarted since it doesn't have sufficient heap space.

It feels like at the very least, consumers should discard buffered records
beyond a certain memory limit and continue to consume and discard until
commit markers. And then refetch committed records. In most cases, this
would fallback to current behavior  providing low latency, but it would
ensure that memory usage is limited and all logs are consumable.


11 b) How will the new buffer.memory being introduced in KIP-81 to control
consumer memory usage be used in the transactional scenario? Would it be
completely ignored? We need to ensure that we don't stop reading from
sockets while waiting for a commit to complete.


12. How will normal log deletion work? Presumably we can have log segments
which contain partial transactions. What happens when that log segment was
deleted (expired)? New consumers replaying logs start consuming partial
transactions?


13. From the discussion thread, I gather that the KIP is based on real use
cases. The KIP does feel very useful. Even so, for some of us who come from
a more traditional messaging background, it will be very useful to
understand the use cases. Can a few be added to the KIP?


14. I think this was mentioned in the doc, but not clear from the KIP.

14a) To use transactions, you need unclean.leader.election=false and
min.insync.replicas > 1 in the broker and producer acks=all - perhaps any
other value with transaction.app.id ought to to be an invalid config for
the producer?

14b) Javadoc in the KIP says producer.app.id - should be transaction.app.id?

14c) To understand the data flow in the KIP, I had to go to the doc to
figure out which flows wait for replication to complete. I did find the
information in the docs, but it will be good to include this in the KIP.


15) As others have mentioned earlier, it may be good to consider a
higher level of durability for transactions. Perhaps for a later KIP, but
it looks like only the messages from the transaction coordinator need to be
flushed to disk to provide that guarantee?





On Mon, Dec 12, 2016 at 6:33 AM, Becket Qin <becket....@gmail.com> wrote:

> Thanks for the KIP. It looks I am late to the party :)
>
> I just went through the KIP and doc for one pass. Some quick feedback:
>
>
>    1.
>
>    UnrecognizedMessageException is a synchronous exception? It is good for
>    it to catch the attention, but seems not intuitive because the
> exception is
>    not caused by the message users are currently sending.
>    2.
>
>    We probably want to have the new protocols and message formats to be
>    added to the wiki, at least a separate link to a dedicated google doc.
> The
>    description of some of the new fields seem missing.
>    3.
>
>    It has been a pain in many cases that we do not know the number of
>    messages in a message set, not sure if the OffsetDelta field in the
> wrapper
>    message will address this.
>    4. Maybe it worth clarifying the targeting topology, does the
>    transaction persists after mirrored? Do the consumer and producer of the
>    consume-process-produce loop have to talk to the same cluster?
>    5.
>
>    It would be good to give one concrete producing example involves a
>    transaction
>    6.
>
>    It seems that if there is one long transaction (in terms of the offsets
>    span, but not number of messages), the consumer side memory pressure
> would
>    be big because we do not allow LCO to go beyond LSO.
>    7.
>
>    What happens if a consumer starts up and seeks to the middle of a
>    transaction?
>
>
> Thanks,
>
> Jiangjie (Becket) Qin
>
> On Sun, Dec 11, 2016 at 5:15 PM, Neha Narkhede <n...@confluent.io> wrote:
>
> > Apurva and Jason -- appreciate the detailed explanation, thank you!
> >
> > Jason -
> >
> > > In many applications, there is already a
> > natural identifier which is already used to divide the workload. For
> > example, in Kafka Streams and Kafka Connect, we have a taskId. For
> > applications where there is no natural ID, the user can generate a UUID
> and
> > persist it locally, which is as good as having the server generate it
> >
> > I think for stateful applications that aren't Streams or Connect, it is
> > more work to leverage the strongest guarantees that this proposal offers.
> > But as you say, stateless applications, on the other hand, can still get
> a
> > lot of it by getting deduplication as well as transactional semantics for
> > the lifetime of the producer instance. I think this is fine since I think
> > that the vast majority of producer apps are stateless and will benefit
> from
> > the stronger guarantees. And for the minority of apps that need to
> > carefully use the application id can just fall back to using Streams
> > instead and expressing the send as a simple Streams topology, if needed.
> >
> > I have a few comments on the naming --
> >
> > 1. I think we should consider renaming initTransactions to just init()
> and
> > moving the metadata initialization there. Let's make sure we don't add
> APIs
> > that are relevant to this proposal only. Instead, try to think what we'd
> > propose if we were writing the producer from scratch today. I suspect we
> > would end up with an init() API that would do the metadata initialization
> > as well as the transaction stuff lazily. If so, let's make that change
> now.
> > 2. Along the same lines, let's think about the role of each id that the
> > producer will have and see if everything still makes sense. For instance,
> > we have quite a few per-producer-instance notions -- client.id, a
> producer
> > id and a transaction.app.id, some set via config and some generated
> > on-the-fly. What role does each play, how do they relate to each other
> and
> > is there an opportunity to get rid of any.
> > 3. I think we should definitely consider renaming transaction.app.id to
> > something else. Given that we already have a notion of application.id
> and
> > it represents the entire Streams application, having transaction.app.id
> > that represents a producer instance is confusing. I do understand that,
> for
> > Streams, the user doesn't have to set transaction.app.id as it will
> likely
> > be application.id+taskId (am I understanding that correctly?)
> >
> > On Fri, Dec 9, 2016 at 2:59 PM Guozhang Wang <wangg...@gmail.com> wrote:
> >
> > Onur,
> >
> > I understand your question now. So it is indeed possible that after
> > commitTxn() returned the messages could still be lost permanently if all
> > replicas failed before the data was flushed to disk. This is the virtue
> of
> > Kafka's design to reply on replication (probably in memory) for high
> > availability, hence async flushing. This scenario already exist today and
> > KIP-98 did not intend to change this factor in any ways.
> >
> > Guozhang
> >
> >
> > On Fri, Dec 9, 2016 at 12:25 PM, Onur Karaman <
> > onurkaraman.apa...@gmail.com>
> > wrote:
> >
> > > In other words, we can see inconsistency when the transaction log
> reports
> > > the transaction as COMMITTED while the markers and data corresponding
> to
> > > the transaction itself on the user partitions may have been partially
> > lost
> > > after-the-fact because of kafka's durability guarantees.
> > >
> > > On Fri, Dec 9, 2016 at 12:16 PM, Onur Karaman <
> > > onurkaraman.apa...@gmail.com>
> > > wrote:
> > >
> > > > @Guozhang no I actually meant durability concerns over COMMIT/ABORT
> > > > markers (and a subset of the user's data produced in the transaction
> > for
> > > > that matter) getting lost from the delta between the write and flush.
> > > >
> > > > KIP-98 relies on replicas writing to logs, so transaction durability
> is
> > > > effectively limited by kafka's definition of a "write success"
> meaning
> > > > written but not flushed to disk.
> > > >
> > > > I mentioned RF=1 not because of availability but actually to
> highlight
> > a
> > > > corner-case durability scenario where the single replica
> participating
> > in
> > > > the transaction experiences a hard failure after the write but before
> > the
> > > > flush, causing the transaction to have partial data loss.
> > > >
> > > > Is this level of durability okay or do we want stronger guarantees
> for
> > > the
> > > > transaction? Basically what I'm wondering is if KIP-98 necessitates
> > > kafka'a
> > > > definition of a "write success" to be extended from "written" to an
> > > > optional "written and flushed to disk".
> > > >
> > > > On Fri, Dec 9, 2016 at 11:54 AM, Michael Pearce <
> michael.pea...@ig.com
> > >
> > > > wrote:
> > > >
> > > >> Apologies on the spelling.
> > > >>
> > > >> *Hi Jay,
> > > >> ________________________________________
> > > >> From: Michael Pearce <michael.pea...@ig.com>
> > > >> Sent: Friday, December 9, 2016 7:52:25 PM
> > > >> To: dev@kafka.apache.org
> > > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > >> Messaging
> > > >>
> > > >> Hi Jey
> > > >>
> > > >> 1) I agree, these should be used to add this in a future kip if ever
> > was
> > > >> enough of a case. As stated for us I think for these systems we will
> > > keep
> > > >> our JMS solutions there.  I think maybe in the docs when this
> feature
> > is
> > > >> written up, one should redirect users to alternative options such as
> > jms
> > > >> brokers, for these use cases.
> > > >>
> > > >> 2) I think this kip needs to be mindful and actually own to make
> sure
> > > >> things are implemented in a way to make future enchancement easy/or
> at
> > > >> least extensible. Having to in future rework things and correct
> > historic
> > > >> decisions is expensive as already finding.
> > > >>
> > > >> Sent using OWA for iPhone
> > > >> ________________________________________
> > > >> From: Jay Kreps <j...@confluent.io>
> > > >> Sent: Friday, December 9, 2016 7:19:59 PM
> > > >> To: dev@kafka.apache.org
> > > >> Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> Transactional
> > > >> Messaging
> > > >>
> > > >> Hey Michael,
> > > >>
> > > >> Yeah, I don't think you need to go into the details of whatever you
> > guys
> > > >> have. I think several people in the thread said "let's do XA
> > > transactions
> > > >> too!" Obviously in a world where features were free and always
> worked
> > > >> perfectly we would! I've probably talked to about 100 people about
> > their
> > > >> use of XA transactions in different systems and my observation has
> > been
> > > >> (a)
> > > >> they are a bit of an operational nightmare, (b) the use cases i've
> > > >> understood don't actually require full XA transactions they actually
> > > >> require a much weaker and easier to guarantee property. The result
> is
> > > you
> > > >> pay a big complexity cost for a guarantee much stronger than what
> you
> > > >> wanted. My sense is that this opinion is broadly shared by the
> > > distributed
> > > >> systems community at large and by Kafka folks in particular.
> > > >>
> > > >> I'm a contrarian so I think it is great not to be too swayed by
> > "common
> > > >> wisdom" though. Five years ago there was a consensus that
> distributed
> > > >> transactions were too hard to implement in an operationally sound
> way,
> > > >> which i think was not correct, so the bad reputation for
> cross-system
> > > >> transactions may be equally wrong!
> > > >>
> > > >> To build a compelling case this is wrong I think two things need to
> be
> > > >> done:
> > > >>
> > > >>    1. Build a case that there are a large/important set of use cases
> > > that
> > > >>    cannot be solved with two independent transactions (as i
> > described),
> > > >> and
> > > >>    that these use cases are things Kafka should be able to do.
> > > >>    2. Come up with the concrete extensions to the KIP-98 proposal
> that
> > > >>    would enable an operationally sound implementation for pluggable
> > > >>    multi-system XA.
> > > >>
> > > >> -Jay
> > > >>
> > > >>
> > > >>
> > > >> On Fri, Dec 9, 2016 at 10:25 AM, Michael Pearce <
> > michael.pea...@ig.com>
> > > >> wrote:
> > > >>
> > > >> > Hi Jay,
> > > >> >
> > > >> > I can't go too deep into exact implantation due to no NDA. So
> > > apologies
> > > >> > here.
> > > >> >
> > > >> > Essentially we have multiple processes each owning selection of
> > > accounts
> > > >> > so on general flows an action for an account just needs to be
> > managed
> > > >> local
> > > >> > to the owning node, happy days ever change is handled as a tick
> tock
> > > >> change.
> > > >> >
> > > >> > Unfortunately when a transfer occurs we need the two processes to
> > > >> > co-ordinate their transaction, we also need to ensure both don't
> > > >> continue
> > > >> > other actions/changesl, we do this using a data grid technology.
> > This
> > > >> grid
> > > >> > technology supports transaction manager that we couple into
> > currently
> > > >> our
> > > >> > jms provider which supports full XA transactions as such we can
> > manage
> > > >> the
> > > >> > production of the change messages out the system transactionally
> as
> > > >> well as
> > > >> > the in grid state.
> > > >> >
> > > >> > The obvious arguement here is should we even look to move this
> flow
> > > off
> > > >> > JMS then. We prob shouldn't nor will do this.
> > > >> >
> > > >> > The point is that I think saying Kafka supports transactions but
> > then
> > > >> not
> > > >> > supporting it as per the traditional sense leads to developers
> > > expecting
> > > >> > similar behaviour and will cause issues in prod when they find it
> > > >> doesn't
> > > >> > work as they're used to.
> > > >> >
> > > >> > As my other response earlier, is there a better name to describe
> > this
> > > >> > feature, if we're not implementing transactions to the traditional
> > > >> > transaction expected, to avoid this confusion?
> > > >> >
> > > >> >
> > > >> > Sent using OWA for iPhone
> > > >> > ________________________________________
> > > >> > From: Jay Kreps <j...@confluent.io>
> > > >> > Sent: Friday, December 9, 2016 6:08:07 PM
> > > >> > To: dev@kafka.apache.org
> > > >> > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > Transactional
> > > >> > Messaging
> > > >> >
> > > >> > Hey Michael,
> > > >> >
> > > >> > Doesn't that example have more to do with applying the update
> > against
> > > >> two
> > > >> > rows in a single transaction? That is, clearly the write to Kafka
> > > needs
> > > >> to
> > > >> > be "transactional" and the write to the destination needs to be
> > > >> > transactional, but it's not clear to me that you need isolation
> that
> > > >> spans
> > > >> > both operations. Can you dive into the system architecture a bit
> > more
> > > >> and
> > > >> > explain why Kafka needs to participate in the same transaction as
> > the
> > > >> > destination system?
> > > >> >
> > > >> > -Jay
> > > >> >
> > > >> > On Thu, Dec 8, 2016 at 10:19 PM, Michael Pearce <
> > > michael.pea...@ig.com>
> > > >> > wrote:
> > > >> >
> > > >> > > Usecase in IG:
> > > >> > >
> > > >> > > Fund transfer between accounts. When we debit one account and
> fund
> > > >> > another
> > > >> > > we must ensure the records to both occur as an acid action, and
> as
> > a
> > > >> > single
> > > >> > > transaction.
> > > >> > >
> > > >> > > Today we achieve this because we have jms, as such we can do the
> > > >> actions
> > > >> > > needed in an xa transaction across both the accounts. To move
> this
> > > >> flow
> > > >> > to
> > > >> > > Kafka we would need support of XA transaction.
> > > >> > >
> > > >> > >
> > > >> > >
> > > >> > > Sent using OWA for iPhone
> > > >> > > ________________________________________
> > > >> > > From: Michael Pearce <michael.pea...@ig.com>
> > > >> > > Sent: Friday, December 9, 2016 6:09:06 AM
> > > >> > > To: dev@kafka.apache.org
> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > > Transactional
> > > >> > > Messaging
> > > >> > >
> > > >> > > Hi Jay,
> > > >> > >
> > > >> > > For me having an XA transaction allows for ensuring ACID across
> my
> > > >> > > application.
> > > >> > >
> > > >> > > I believe it is part of the JMS api, and obviously JMS still is
> in
> > > >> > > enterprise very widely adopted for Messaging transport , so
> > > obviously
> > > >> to
> > > >> > > say it isn't widely used i think is ignoring a whole range of
> > users.
> > > >> Like
> > > >> > > wise I believe frameworks like spring etc fully support it more
> > > >> evidence
> > > >> > of
> > > >> > > its wide adoption.
> > > >> > >
> > > >> > > On this note personally we try to avoid transactions entirely in
> > our
> > > >> > flows
> > > >> > > for performance and simplicity. but we do alas unfortunately
> have
> > > one
> > > >> or
> > > >> > > two places we cannot ignore it.
> > > >> > >
> > > >> > > Cheers
> > > >> > > Mike
> > > >> > >
> > > >> > > Sent using OWA for iPhone
> > > >> > > ________________________________________
> > > >> > > From: Jay Kreps <j...@confluent.io>
> > > >> > > Sent: Thursday, December 8, 2016 11:25:53 PM
> > > >> > > To: dev@kafka.apache.org
> > > >> > > Subject: Re: [DISCUSS] KIP-98: Exactly Once Delivery and
> > > Transactional
> > > >> > > Messaging
> > > >> > >
> > > >> > > Hey Edoardo,
> > > >> > >
> > > >> > > For (3) can you outline what you think the benefit and use cases
> > > for a
> > > >> > more
> > > >> > > general cross-system XA feature would be an what changes to the
> > > >> proposal
> > > >> > > would be required to enable it? When I have asked people who
> > wanted
> > > >> > > cross-system XA in the past what they wanted it for, I haven't
> > > really
> > > >> > > gotten any answers that made sense. Every person really wanted
> > > >> something
> > > >> > > that would be better solved by a transactional (or idempotent)
> > write
> > > >> to
> > > >> > > Kafka followed by an independent transactional (or idempotent)
> > > >> > consumption
> > > >> > > (which this proposal enables). For the use cases they described
> > > tying
> > > >> > these
> > > >> > > two things together had no advantage and many disadvantages.
> > > >> > >
> > > >> > > I have one use case which would be accomplished by cross-system
> XA
> > > >> which
> > > >> > is
> > > >> > > allowing the producer to block on the synchronous processing of
> > the
> > > >> > message
> > > >> > > by (all? some?) consumers. However I'm not convinced that
> > > >> cross-system XA
> > > >> > > is the best solution to this problem, and I'm also not convinced
> > > this
> > > >> is
> > > >> > an
> > > >> > > important problem to solve. But maybe you have something in mind
> > > here.
> > > >> > >
> > > >> > > -Jay
> > > >> > >
> > > >> > >
> > > >> > >
> > > >> > > On Thu, Dec 8, 2016 at 1:15 PM, Edoardo Comar <
> eco...@uk.ibm.com>
> > > >> wrote:
> > > >> > >
> > > >> > > > Hi,
> > > >> > > > thanks, very interesting KIP ... I haven't fully digested it
> > yet.
> > > >> > > >
> > > >> > > > We have many users who choose not to use the Java client,  so
> I
> > > have
> > > >> > > > concerns about the added complexity in developing the clients.
> > > >> > > > A few questions.
> > > >> > > >
> > > >> > > > 1 - is mixing transactional and non transactional messages on
> > the
> > > >> *same
> > > >> > > > topic-partition* really a requirement ?
> > > >> > > > What use case does it satisfy?
> > > >> > > >
> > > >> > > > 2 - I guess some clients may only be interested to implement
> the
> > > >> > producer
> > > >> > > > idempotency.
> > > >> > > > It's not clear how they could be implemented without having to
> > add
> > > >> the
> > > >> > > > transaction capabilities.
> > > >> > > > As others on this list have said, I too would like to see
> > > >> idempotency
> > > >> > as
> > > >> > > a
> > > >> > > > more basic feature, on top which txns can be built.
> > > >> > > >
> > > >> > > > 3 - The KIP seems focused on a use case where consumption
> from a
> > > >> topic
> > > >> > > and
> > > >> > > > subsequent production are part of the producer transaction.
> > > >> > > >
> > > >> > > > It'd be great to see a way to extend the producer transaction
> to
> > > >> > include
> > > >> > > > additional transactional resources,
> > > >> > > > so that the consumption from another topic just becomes a
> > special
> > > >> case
> > > >> > of
> > > >> > > > a more general "distributed" txn.
> > > >> > > >
> > > >> > > > Edo
> > > >> > > > --------------------------------------------------
> > > >> > > > Edoardo Comar
> > > >> > > > IBM MessageHub
> > > >> > > > eco...@uk.ibm.com
> > > >> > > > IBM UK Ltd, Hursley Park, SO21 2JN
> > > >> > > >
> > > >> > > > IBM United Kingdom Limited Registered in England and Wales
> with
> > > >> number
> > > >> > > > 741598 Registered office: PO Box 41, North Harbour,
> Portsmouth,
> > > >> Hants.
> > > >> > > PO6
> > > >> > > > 3AU
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > From:   Guozhang Wang <wangg...@gmail.com>
> > > >> > > > To:     "dev@kafka.apache.org" <dev@kafka.apache.org>
> > > >> > > > Date:   30/11/2016 22:20
> > > >> > > > Subject:        [DISCUSS] KIP-98: Exactly Once Delivery and
> > > >> > Transactional
> > > >> > > > Messaging
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > Hi all,
> > > >> > > >
> > > >> > > > I have just created KIP-98 to enhance Kafka with exactly once
> > > >> delivery
> > > >> > > > semantics:
> > > >> > > >
> > > >> > > > *https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >> > > > <
> > > >> > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > >> > > > 98+-+Exactly+Once+Delivery+and+Transactional+Messaging
> > > >> > > > >*
> > > >> > > >
> > > >> > > > This KIP adds a transactional messaging mechanism along with
> an
> > > >> > > idempotent
> > > >> > > > producer implementation to make sure that 1) duplicated
> messages
> > > >> sent
> > > >> > > from
> > > >> > > > the same identified producer can be detected on the broker
> side,
> > > and
> > > >> > 2) a
> > > >> > > > group of messages sent within a transaction will atomically be
> > > >> either
> > > >> > > > reflected and fetchable to consumers or not as a whole.
> > > >> > > >
> > > >> > > > The above wiki page provides a high-level view of the proposed
> > > >> changes
> > > >> > as
> > > >> > > > well as summarized guarantees. Initial draft of the detailed
> > > >> > > > implementation
> > > >> > > > design is described in this Google doc:
> > > >> > > >
> > > >> > > > https://docs.google.com/document/d/11Jqy_
> > > >> > GjUGtdXJK94XGsEIK7CP1SnQGdp2eF
> > > >> > > > 0wSw9ra8
> > > >> > > >
> > > >> > > >
> > > >> > > > We would love to hear your comments and suggestions.
> > > >> > > >
> > > >> > > > Thanks,
> > > >> > > >
> > > >> > > > -- Guozhang
> > > >> > > >
> > > >> > > >
> > > >> > > >
> > > >> > > > Unless stated otherwise above:
> > > >> > > > IBM United Kingdom Limited - Registered in England and Wales
> > with
> > > >> > number
> > > >> > > > 741598.
> > > >> > > > Registered office: PO Box 41, North Harbour, Portsmouth,
> > Hampshire
> > > >> PO6
> > > >> > > 3AU
> > > >> > > >
> > > >> > > The information contained in this email is strictly confidential
> > and
> > > >> for
> > > >> > > the use of the addressee only, unless otherwise indicated. If
> you
> > > are
> > > >> not
> > > >> > > the intended recipient, please do not read, copy, use or
> disclose
> > to
> > > >> > others
> > > >> > > this message or any attachment. Please also notify the sender by
> > > >> replying
> > > >> > > to this email or by telephone (+44(020 7896 0011) and then
> delete
> > > the
> > > >> > email
> > > >> > > and any copies of it. Opinions, conclusion (etc) that do not
> > relate
> > > to
> > > >> > the
> > > >> > > official business of this company shall be understood as neither
> > > given
> > > >> > nor
> > > >> > > endorsed by it. IG is a trading name of IG Markets Limited (a
> > > company
> > > >> > > registered in England and Wales, company number 04008957) and IG
> > > Index
> > > >> > > Limited (a company registered in England and Wales, company
> number
> > > >> > > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> > > Hill,
> > > >> > > London EC4R 2YA. Both IG Markets Limited (register number
> 195355)
> > > and
> > > >> IG
> > > >> > > Index Limited (register number 114059) are authorised and
> > regulated
> > > by
> > > >> > the
> > > >> > > Financial Conduct Authority.
> > > >> > >
> > > >> > The information contained in this email is strictly confidential
> and
> > > for
> > > >> > the use of the addressee only, unless otherwise indicated. If you
> > are
> > > >> not
> > > >> > the intended recipient, please do not read, copy, use or disclose
> to
> > > >> others
> > > >> > this message or any attachment. Please also notify the sender by
> > > >> replying
> > > >> > to this email or by telephone (+44(020 7896 0011) and then delete
> > the
> > > >> email
> > > >> > and any copies of it. Opinions, conclusion (etc) that do not
> relate
> > to
> > > >> the
> > > >> > official business of this company shall be understood as neither
> > given
> > > >> nor
> > > >> > endorsed by it. IG is a trading name of IG Markets Limited (a
> > company
> > > >> > registered in England and Wales, company number 04008957) and IG
> > Index
> > > >> > Limited (a company registered in England and Wales, company number
> > > >> > 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> > Hill,
> > > >> > London EC4R 2YA. Both IG Markets Limited (register number 195355)
> > and
> > > IG
> > > >> > Index Limited (register number 114059) are authorised and
> regulated
> > by
> > > >> the
> > > >> > Financial Conduct Authority.
> > > >> >
> > > >> The information contained in this email is strictly confidential and
> > for
> > > >> the use of the addressee only, unless otherwise indicated. If you
> are
> > > not
> > > >> the intended recipient, please do not read, copy, use or disclose to
> > > others
> > > >> this message or any attachment. Please also notify the sender by
> > > replying
> > > >> to this email or by telephone (+44(020 7896 0011) and then delete
> the
> > > email
> > > >> and any copies of it. Opinions, conclusion (etc) that do not relate
> to
> > > the
> > > >> official business of this company shall be understood as neither
> given
> > > nor
> > > >> endorsed by it. IG is a trading name of IG Markets Limited (a
> company
> > > >> registered in England and Wales, company number 04008957) and IG
> Index
> > > >> Limited (a company registered in England and Wales, company number
> > > >> 01190902). Registered address at Cannon Bridge House, 25 Dowgate
> Hill,
> > > >> London EC4R 2YA. Both IG Markets Limited (register number 195355)
> and
> > IG
> > > >> Index Limited (register number 114059) are authorised and regulated
> by
> > > the
> > > >> Financial Conduct Authority.
> > > >>
> > > >
> > > >
> > >
> >
> >
> >
> > --
> > -- Guozhang
> >
> > --
> > Thanks,
> > Neha
> >
>



-- 
Regards,

Rajini

Reply via email to