Thanks Jason, I reviewed the down-converting logic following your
explanation.[^1] You mean the following routines, right?

-
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/server/KafkaApis.scala#L534
-
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/record/LazyDownConversionRecords.java#L165
-
https://github.com/apache/kafka/blob/trunk/clients/src/main/java/org/apache/kafka/common/record/RecordsUtil.java#L40

It seems like your stance is like following:

1. In principle, Kafka does not change the compression codec when
down-converting, since it requires inspecting the fetched data, which is
expensive.
2. However, there are some cases the fetched data is inspected anyway. In
this case, we can provide compression conversion from Zstandard to
classical ones[^2].

And from what I understand, the cases where the client without ZStandard
support receives ZStandard compressed records can be organized into two
cases:

a. The 'compression.type' configuration of given topic is 'producer' and
the producer compressed the records with ZStandard. (that is, using
ZStandard implicitly.)
b.  The 'compression.type' configuration of given topic is 'zstd'; that is,
using ZStandard explicitly.

As you stated, we don't have to handle the case b specially. So, It seems
like we can narrow the focus of the problem by joining case 1 and case b
like the following:

> Given the topic with 'producer' as its 'compression.type' configuration,
ZStandard compressed records and old client without ZStandard, is there any
case we need to inspect the records and can change the compression type? If
so, can we provide compression type converting?

Do I understand correctly?

Best,
Dongjin

[^1]: I'm sorry, I found that I was a little bit misunderstanding how API
version works, after reviewing the downconvert logic & the protocol
documentation <https://kafka.apache.org/protocol>.
[^2]: None, Gzip, Snappy, Lz4.

On Tue, Aug 14, 2018 at 2:16 AM Jason Gustafson <ja...@confluent.io> wrote:

> >
> > But in my opinion, since the client will fail with the API version, so we
> > don't need to down-convert the messages anyway. Isn't it? So, I think we
> > don't care about this case. (I'm sorry, I am not familiar with
> down-convert
> > logic.)
>
>
> Currently the broker down-converts automatically when it receives an old
> version of the fetch request (a version which is known to predate the
> message format in use). Typically when down-converting the message format,
> we use the same compression type, but there is not much point in doing so
> when we know the client doesn't support it. So if zstandard is in use, and
> we have to down-convert anyway, then we can choose to use a different
> compression type or no compression type.
>
> From my perspective, there is no significant downside to bumping the
> protocol version and it has several potential benefits. Version bumps are
> cheap. The main question mark in my mind is about down-conversion. Figuring
> out whether down-conversion is needed is hard generally without inspecting
> the fetched data, which is expensive. I think we agree in principle that we
> do not want to have to pay this cost generally and prefer the clients to
> fail when they see an unhandled compression type. The point I was making is
> that there are some cases where we are either inspecting the data anyway
> (because we have to down-convert the message format), or we have an easy
> way to tell whether zstandard is in use (the topic has it configured
> explicitly). In the latter case, we don't have to handle it specially. But
> we do have to decide how we will handle down-conversion to older formats.
>
> -Jason
>
> On Sun, Aug 12, 2018 at 5:15 PM, Dongjin Lee <dong...@apache.org> wrote:
>
> > Colin and Jason,
> >
> > Thanks for your opinions. In summarizing, the Pros and Cons of bumping
> > fetch API version are:
> >
> > Cons:
> >
> > - The Broker can't know whether a given message batch is compressed with
> > zstd or not.
> > - Need some additional logic for the topic explicitly configured to use
> > zstd.
> >
> > Pros:
> >
> > - The broker doesn't need to conduct expensive down-conversion.
> > - Can message the users to update their client.
> >
> > So, opinions for the backward-compatibility policy by far:
> >
> > - A: bump the API version - +2 (Colin, Jason)
> > - B: leave unchanged - +1 (Viktor)
> >
> > Here are my additional comments:
> >
> > @Colin
> >
> > I greatly appreciate your response. In the case of the dictionary
> support,
> > of course, this issue should be addressed later so we don't need it in
> the
> > first version. You are right - it is not late to try it after some
> > benchmarks. What I mean is, we should keep in mind on that potential
> > feature.
> >
> > @Jason
> >
> > You wrote,
> >
> > > Similarly, if we have to down-convert anyway because the client does
> not
> > understand the message format, then we could also use a different
> > compression type.
> >
> > But in my opinion, since the client will fail with the API version, so we
> > don't need to down-convert the messages anyway. Isn't it? So, I think we
> > don't care about this case. (I'm sorry, I am not familiar with
> down-convert
> > logic.)
> >
> > Please give more opinions. Thanks!
> >
> > - Dongjin
> >
> >
> > On Wed, Aug 8, 2018 at 6:41 AM Jason Gustafson <ja...@confluent.io>
> wrote:
> >
> > > Hey Colin,
> > >
> > > The problem for the fetch API is that the broker does not generally
> know
> > if
> > > a batch was compressed with zstd unless it parses it. I think the goal
> > here
> > > is to avoid the expensive down-conversion that is needed to ensure
> > > compatibility because it is only necessary if zstd is actually in use.
> > But
> > > as long as old clients can parse the message format, they should get a
> > > reasonable error if they see an unsupported compression type in the
> > > attributes. Basically the onus is on users to ensure that their
> consumers
> > > have been updated prior to using zstd. It seems like a reasonable
> > tradeoff
> > > to me. There are a couple cases that might be worth thinking through:
> > >
> > > 1. If a topic is explicitly configured to use zstd, then we don't need
> to
> > > check the fetched data for the compression type to know if we need
> > > down-conversion. If we did bump the Fetch API version, then we could
> > handle
> > > this case by either down-converting using a different compression type
> or
> > > returning an error.
> > > 2. Similarly, if we have to down-convert anyway because the client does
> > not
> > > understand the message format, then we could also use a different
> > > compression type.
> > >
> > > For the produce API, I think it's reasonable to bump the api version.
> > This
> > > can be used by clients to check whether a broker supports zstd. For
> > > example, we might support a list of preferred compression types in the
> > > producer and we could use the broker to detect which version to use.
> > >
> > > -Jason
> > >
> > > On Tue, Aug 7, 2018 at 1:32 PM, Colin McCabe <cmcc...@apache.org>
> wrote:
> > >
> > > > Thanks for bumping this, Dongjin.  ZStd is a good compression codec
> > and I
> > > > hope we can get this support in soon!
> > > >
> > > > I would say we can just bump the API version to indicate that ZStd
> > > support
> > > > is expected in new clients.  We probably need some way of indicating
> to
> > > the
> > > > older clients that they can't consume the partitions, as well.
> Perhaps
> > > we
> > > > can use the UNSUPPORTED_FOR_MESSAGE_FORMAT error?
> > > >
> > > > The license thing seems straightforward -- it's just a matter of
> adding
> > > > the text to the right files as per ASF guidelines.
> > > >
> > > > With regard to the dictionary support, do we really need that in the
> > > first
> > > > version?  Hopefully message batches are big enough that this isn't
> > > needed.
> > > > Some benchmarks might help here.
> > > >
> > > > best,
> > > > Colin
> > > >
> > > >
> > > > On Tue, Aug 7, 2018, at 08:02, Dongjin Lee wrote:
> > > > > As Kafka 2.0.0 was released, let's reboot this issue, KIP-110
> > > > > <https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 110%3A+Add+Codec+for+ZStandard+Compression>
> > > > > .
> > > > >
> > > > > For newcomers, Here is some summary of the history: KIP-110 was
> > > > originally
> > > > > worked for the issue KAFKA-4514 but, it lacked benchmark results to
> > get
> > > > the
> > > > > agreement of the community. Later, Ivan Babrou and some other users
> > who
> > > > > adopted the patch provided their excellent performance report which
> > is
> > > > now
> > > > > included in the KIP, but it postponed again because of the
> community
> > > was
> > > > > busy for 2.0.0 release. It is why I now reboot this issue.
> > > > >
> > > > > The following is the current status of the feature: You can check
> the
> > > > > current draft implementation here
> > > > > <https://github.com/apache/kafka/pull/2267>. It is based on zstd
> > 1.3.5
> > > > and
> > > > > periodically rebased onto the latest trunk[^1].
> > > > >
> > > > > The issues that should be addressed is like following:
> > > > >
> > > > > *1. Backward Compatibility*
> > > > >
> > > > > To support old consumers, we need to take a strategy to handle the
> > old
> > > > > consumers. Current candidates are:
> > > > >
> > > > > - Bump API version
> > > > > - Leave unchanged: let the old clients fail.
> > > > > - Improve the error messages:
> > > > >
> > > > > *2. Dictionary Support*
> > > > >
> > > > > To support zstd's dictionary feature in the future (if needed), we
> > need
> > > > to
> > > > > sketch how it should be and leave some room for it. As of now,
> there
> > > has
> > > > > been no discussion on this topic yet.
> > > > >
> > > > > *3. License*
> > > > >
> > > > > To use this feature, we need to add license of zstd and zstd-jni to
> > the
> > > > > project. (Thanks to Viktor Somogyi for raising this issue!) It
> seems
> > > like
> > > > > what Apache Spark did would be a good example but there has been no
> > > > > discussion yet.
> > > > >
> > > > > You can find the details of the above issues in the KIP document.
> > > Please
> > > > > have a look when you are free, and give me feedback. All kinds of
> > > > > participating are welcome.
> > > > >
> > > > > Best,
> > > > > Dongjin
> > > > >
> > > > > [^1]: At the time of writing, commit 6b4fb8152.
> > > > >
> > > > > On Sat, Jul 14, 2018 at 10:45 PM Dongjin Lee <dong...@apache.org>
> > > wrote:
> > > > >
> > > > > > Sorry for the late reply.
> > > > > >
> > > > > > In short, I could not submit the updated KIP by the feature
> freeze
> > > > > > deadline of 2.0.0. For this reason, it will not be included in
> the
> > > > 2.0.0
> > > > > > release and all discussion for this issue were postponed after
> the
> > > > release
> > > > > > of 2.0.0.
> > > > > >
> > > > > > I have been updating the PR following recent updates. Just now, I
> > > > rebased
> > > > > > it against the latest trunk and updated the zstd version into
> > 1.3.5.
> > > > If you
> > > > > > need some request, don't hesitate to notify me. (But not this
> > thread
> > > -
> > > > just
> > > > > > send me the message directly.)
> > > > > >
> > > > > > Best,
> > > > > > Dongjin
> > > > > >
> > > > > > On Tue, Jul 10, 2018 at 11:57 PM Bobby Evans <bo...@apache.org>
> > > wrote:
> > > > > >
> > > > > >> I there any update on this.  The performance improvements are
> > quite
> > > > > >> impressive and I really would like to stop forking kafka just to
> > get
> > > > this
> > > > > >> in.
> > > > > >>
> > > > > >> Thanks,
> > > > > >>
> > > > > >> Bobby
> > > > > >>
> > > > > >> On Wed, Jun 13, 2018 at 8:56 PM Dongjin Lee <dong...@apache.org
> >
> > > > wrote:
> > > > > >>
> > > > > >> > Ismael,
> > > > > >> >
> > > > > >> > Oh, I forgot all of you are on working frenzy for 2.0! No
> > problem,
> > > > take
> > > > > >> > your time. I am also working at another issue now. Thank you
> for
> > > > > >> letting me
> > > > > >> > know.
> > > > > >> >
> > > > > >> > Best,
> > > > > >> > Dongjin
> > > > > >> >
> > > > > >> > On Wed, Jun 13, 2018, 11:44 PM Ismael Juma <isma...@gmail.com
> >
> > > > wrote:
> > > > > >> >
> > > > > >> > > Sorry for the delay Dongjin. Everyone is busy finalising
> > 2.0.0.
> > > > This
> > > > > >> KIP
> > > > > >> > > seems like a great candidate for 2.1.0 and hopefully there
> > will
> > > be
> > > > > >> more
> > > > > >> > of
> > > > > >> > > a discussion next week. :)
> > > > > >> > >
> > > > > >> > > Ismael
> > > > > >> > >
> > > > > >> > > On Wed, 13 Jun 2018, 05:17 Dongjin Lee, <dong...@apache.org
> >
> > > > wrote:
> > > > > >> > >
> > > > > >> > > > Hello. I just updated my draft implementation:
> > > > > >> > > >
> > > > > >> > > > 1. Rebased to latest trunk (commit 5145d6b)
> > > > > >> > > > 2. Apply ZStd 1.3.4
> > > > > >> > > >
> > > > > >> > > > You can check out the implementation from here
> > > > > >> > > > <https://github.com/apache/kafka/pull/2267>. If you
> > > experience
> > > > any
> > > > > >> > > problem
> > > > > >> > > > running it, don't hesitate to give me a mention.
> > > > > >> > > >
> > > > > >> > > > Best,
> > > > > >> > > > Dongjin
> > > > > >> > > >
> > > > > >> > > > On Tue, Jun 12, 2018 at 6:50 PM Dongjin Lee <
> > > dong...@apache.org
> > > > >
> > > > > >> > wrote:
> > > > > >> > > >
> > > > > >> > > > > Here is the short conclusion about the license problem:
> > *We
> > > > can
> > > > > >> use
> > > > > >> > > zstd
> > > > > >> > > > > and zstd-jni without any problem, but we need to include
> > > their
> > > > > >> > license,
> > > > > >> > > > > e.g., BSD license.*
> > > > > >> > > > >
> > > > > >> > > > > Both of BSD 2 Clause License & 3 Clause License requires
> > to
> > > > > >> include
> > > > > >> > the
> > > > > >> > > > > license used, and BSD 3 Clause License requires that the
> > > name
> > > > of
> > > > > >> the
> > > > > >> > > > > contributor can't be used to endorse or promote the
> > product.
> > > > > >> That's
> > > > > >> > it
> > > > > >> > > > > <
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >> http://www.mikestratton.net/2011/12/is-bsd-license-
> > > > compatible-with-apache-2-0-license/
> > > > > >> > > > >
> > > > > >> > > > > - They are not listed in the list of prohibited licenses
> > > > > >> > > > > <https://www.apache.org/legal/resolved.html#category-x>
> > > also.
> > > > > >> > > > >
> > > > > >> > > > > Here is how Spark did for it
> > > > > >> > > > > <https://issues.apache.org/jira/browse/SPARK-19112>:
> > > > > >> > > > >
> > > > > >> > > > > - They made a directory dedicated to the dependency
> > license
> > > > files
> > > > > >> > > > > <https://github.com/apache/spark/tree/master/licenses>
> > and
> > > > added
> > > > > >> > > > licenses
> > > > > >> > > > > for Zstd
> > > > > >> > > > > <
> > > > > >> > https://github.com/apache/spark/blob/master/licenses/
> > > > LICENSE-zstd.txt
> > > > > >> > > >
> > > > > >> > > > &
> > > > > >> > > > > Zstd-jni
> > > > > >> > > > > <
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >> https://github.com/apache/spark/blob/master/licenses/
> > > > LICENSE-zstd-jni.txt
> > > > > >> >
> > > > > >> > > > > .
> > > > > >> > > > > - Added a link to the original license files in LICENSE.
> > > > > >> > > > > <https://github.com/apache/spark/pull/18805/files>
> > > > > >> > > > >
> > > > > >> > > > > If needed, I can make a similar update.
> > > > > >> > > > >
> > > > > >> > > > > Thanks for pointing out this problem, Viktor! Nice
> catch!
> > > > > >> > > > >
> > > > > >> > > > > Best,
> > > > > >> > > > > Dongjin
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > > On Mon, Jun 11, 2018 at 11:50 PM Dongjin Lee <
> > > > dong...@apache.org>
> > > > > >> > > wrote:
> > > > > >> > > > >
> > > > > >> > > > >> I greatly appreciate your comprehensive reasoning. so:
> +1
> > > > for b
> > > > > >> > until
> > > > > >> > > > now.
> > > > > >> > > > >>
> > > > > >> > > > >> For the license issues, I will have a check on how the
> > over
> > > > > >> projects
> > > > > >> > > are
> > > > > >> > > > >> doing and share the results.
> > > > > >> > > > >>
> > > > > >> > > > >> Best,
> > > > > >> > > > >> Dongjin
> > > > > >> > > > >>
> > > > > >> > > > >> On Mon, Jun 11, 2018 at 10:08 PM Viktor Somogyi <
> > > > > >> > > > viktorsomo...@gmail.com>
> > > > > >> > > > >> wrote:
> > > > > >> > > > >>
> > > > > >> > > > >>> Hi Dongjin,
> > > > > >> > > > >>>
> > > > > >> > > > >>> A couple of comments:
> > > > > >> > > > >>> I would vote for option b. in the "backward
> > compatibility"
> > > > > >> section.
> > > > > >> > > My
> > > > > >> > > > >>> reasoning for this is that users upgrading to a zstd
> > > > compatible
> > > > > >> > > version
> > > > > >> > > > >>> won't start to use it automatically, so manual
> > > > reconfiguration
> > > > > >> is
> > > > > >> > > > >>> required.
> > > > > >> > > > >>> Therefore an upgrade won't mess up the cluster. If not
> > all
> > > > the
> > > > > >> > > clients
> > > > > >> > > > >>> are
> > > > > >> > > > >>> upgraded but just some of them and they'd start to use
> > > zstd
> > > > > >> then it
> > > > > >> > > > would
> > > > > >> > > > >>> cause errors in the cluster. I'd like to presume
> though
> > > that
> > > > > >> this
> > > > > >> > is
> > > > > >> > > a
> > > > > >> > > > >>> very
> > > > > >> > > > >>> obvious failure case and nobody should be surprised if
> > it
> > > > didn't
> > > > > >> > > work.
> > > > > >> > > > >>> I wouldn't choose a. as I think we should bump the
> fetch
> > > and
> > > > > >> > produce
> > > > > >> > > > >>> requests if it's a change in the message format.
> > Moreover
> > > if
> > > > > >> some
> > > > > >> > of
> > > > > >> > > > the
> > > > > >> > > > >>> producers and the brokers are upgraded but some of the
> > > > consumers
> > > > > >> > are
> > > > > >> > > > not,
> > > > > >> > > > >>> then we wouldn't prevent the error when the old
> consumer
> > > > tries
> > > > > >> to
> > > > > >> > > > consume
> > > > > >> > > > >>> the zstd compressed messages.
> > > > > >> > > > >>> I wouldn't choose c. either as I think binding the
> > > > compression
> > > > > >> type
> > > > > >> > > to
> > > > > >> > > > an
> > > > > >> > > > >>> API is not so obvious from the developer's
> perspective.
> > > > > >> > > > >>>
> > > > > >> > > > >>> I would also prefer to use the existing binding,
> however
> > > we
> > > > must
> > > > > >> > > > respect
> > > > > >> > > > >>> the licenses:
> > > > > >> > > > >>> "The code for these JNI bindings is licenced under
> > > 2-clause
> > > > BSD
> > > > > >> > > > license.
> > > > > >> > > > >>> The native Zstd library is licensed under 3-clause BSD
> > > > license
> > > > > >> and
> > > > > >> > > > GPL2"
> > > > > >> > > > >>> Based on the FAQ page
> > > > > >> > > > >>> https://www.apache.org/legal/resolved.html#category-a
> > > > > >> > > > >>> we may use 2- and 3-clause BSD licenses but the Apache
> > > > license
> > > > > >> is
> > > > > >> > not
> > > > > >> > > > >>> compatible with GPL2. I'm hoping that the "3-clause
> BSD
> > > > license
> > > > > >> and
> > > > > >> > > > GPL2"
> > > > > >> > > > >>> is really not an AND but an OR in this case, but I'm
> no
> > > > lawyer,
> > > > > >> > just
> > > > > >> > > > >>> wanted
> > > > > >> > > > >>> to make the point that we should watch out for
> licenses.
> > > :)
> > > > > >> > > > >>>
> > > > > >> > > > >>> Regards,
> > > > > >> > > > >>> Viktor
> > > > > >> > > > >>>
> > > > > >> > > > >>>
> > > > > >> > > > >>> On Sun, Jun 10, 2018 at 3:02 AM Ivan Babrou <
> > > > ibob...@gmail.com>
> > > > > >> > > wrote:
> > > > > >> > > > >>>
> > > > > >> > > > >>> > Hello,
> > > > > >> > > > >>> >
> > > > > >> > > > >>> > This is Ivan and I still very much support the fact
> > that
> > > > zstd
> > > > > >> > > > >>> compression
> > > > > >> > > > >>> > should be included out of the box.
> > > > > >> > > > >>> >
> > > > > >> > > > >>> > Please think about the environment, you can save
> > quite a
> > > > lot
> > > > > >> of
> > > > > >> > > > >>> hardware
> > > > > >> > > > >>> > with it.
> > > > > >> > > > >>> >
> > > > > >> > > > >>> > Thank you.
> > > > > >> > > > >>> >
> > > > > >> > > > >>> > On Sat, Jun 9, 2018 at 14:14 Dongjin Lee <
> > > > dong...@apache.org>
> > > > > >> > > wrote:
> > > > > >> > > > >>> >
> > > > > >> > > > >>> > > Since there are no responses for a week, I decided
> > to
> > > > > >> > reinitiate
> > > > > >> > > > the
> > > > > >> > > > >>> > > discussion thread.
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> >
> > > > > >> > > > >>>
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-
> > > > 110%3A+Add+Codec+for+ZStandard+Compression
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> > > This KIP is about to introduce ZStandard
> Compression
> > > > into
> > > > > >> > Apache
> > > > > >> > > > >>> Kafka.
> > > > > >> > > > >>> > > The reason why it is posted again has a story: It
> > was
> > > > > >> > originally
> > > > > >> > > > >>> posted
> > > > > >> > > > >>> > to
> > > > > >> > > > >>> > > the dev mailing list more than one year ago but
> > since
> > > it
> > > > > >> has no
> > > > > >> > > > >>> > performance
> > > > > >> > > > >>> > > report included, it was postponed later. But Some
> > > people
> > > > > >> > > (including
> > > > > >> > > > >>> Ivan)
> > > > > >> > > > >>> > > reported excellent performance report with the
> draft
> > > PR,
> > > > > >> this
> > > > > >> > > work
> > > > > >> > > > >>> is now
> > > > > >> > > > >>> > > reactivated.
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> > > The updated KIP document includes some expected
> > > > problems and
> > > > > >> > > their
> > > > > >> > > > >>> > > candidate alternatives. Please have a look when
> you
> > > are
> > > > > >> free,
> > > > > >> > and
> > > > > >> > > > >>> give
> > > > > >> > > > >>> > me a
> > > > > >> > > > >>> > > feedback. All kinds of participating are welcome.
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> > > Best,
> > > > > >> > > > >>> > > Dongjin
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> > > --
> > > > > >> > > > >>> > > *Dongjin Lee*
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> > > *A hitchhiker in the mathematical world.*
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> > > *github:  <http://goog_969573159/>github
> > > > .com/dongjinleekr
> > > > > >> > > > >>> > > <http://github.com/dongjinleekr>linkedin:
> > > > > >> > > > >>> > kr.linkedin.com/in/dongjinleekr
> > > > > >> > > > >>> > > <http://kr.linkedin.com/in/dongjinleekr
> >slideshare:
> > > > > >> > > > >>> > www.slideshare.net/dongjinleekr
> > > > > >> > > > >>> > > <http://www.slideshare.net/dongjinleekr>*
> > > > > >> > > > >>> > >
> > > > > >> > > > >>> >
> > > > > >> > > > >>>
> > > > > >> > > > >> --
> > > > > >> > > > >> *Dongjin Lee*
> > > > > >> > > > >>
> > > > > >> > > > >> *A hitchhiker in the mathematical world.*
> > > > > >> > > > >>
> > > > > >> > > > >> *github:  <http://goog_969573159/>github
> > .com/dongjinleekr
> > > > > >> > > > >> <http://github.com/dongjinleekr>linkedin:
> > > > > >> > > > kr.linkedin.com/in/dongjinleekr
> > > > > >> > > > >> <http://kr.linkedin.com/in/dongjinleekr>slideshare:
> > > > > >> > > > www.slideshare.net/dongjinleekr
> > > > > >> > > > >> <http://www.slideshare.net/dongjinleekr>*
> > > > > >> > > > >>
> > > > > >> > > > >
> > > > > >> > > > >
> > > > > >> > > > > --
> > > > > >> > > > > *Dongjin Lee*
> > > > > >> > > > >
> > > > > >> > > > > *A hitchhiker in the mathematical world.*
> > > > > >> > > > >
> > > > > >> > > > > *github:  <http://goog_969573159/>
> github.com/dongjinleekr
> > > > > >> > > > > <http://github.com/dongjinleekr>linkedin:
> > > > > >> > > > kr.linkedin.com/in/dongjinleekr
> > > > > >> > > > > <http://kr.linkedin.com/in/dongjinleekr>slideshare:
> > > > > >> > > > www.slideshare.net/dongjinleekr
> > > > > >> > > > > <http://www.slideshare.net/dongjinleekr>*
> > > > > >> > > > >
> > > > > >> > > >
> > > > > >> > > >
> > > > > >> > > > --
> > > > > >> > > > *Dongjin Lee*
> > > > > >> > > >
> > > > > >> > > > *A hitchhiker in the mathematical world.*
> > > > > >> > > >
> > > > > >> > > > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > > >> > > > <http://github.com/dongjinleekr>linkedin:
> > > > > >> > > kr.linkedin.com/in/dongjinleekr
> > > > > >> > > > <http://kr.linkedin.com/in/dongjinleekr>slideshare:
> > > > > >> > > > www.slideshare.net/dongjinleekr
> > > > > >> > > > <http://www.slideshare.net/dongjinleekr>*
> > > > > >> > > >
> > > > > >> > >
> > > > > >> >
> > > > > >>
> > > > > >> --
> > > > > >> *Dongjin Lee*
> > > > > >>
> > > > > >> *A hitchhiker in the mathematical world.*
> > > > > >>
> > > > > >> *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > > >> <http://github.com/dongjinleekr>linkedin: kr.linkedin.com/in/
> > > > dongjinleekr
> > > > > >> <http://kr.linkedin.com/in/dongjinleekr>slideshare:
> > > > www.slideshare.net/dongjinleekr
> > > > > >> <http://www.slideshare.net/dongjinleekr>*
> > > > > >>
> > > > > >
> > > > >
> > > > > --
> > > > > *Dongjin Lee*
> > > > >
> > > > > *A hitchhiker in the mathematical world.*
> > > > >
> > > > > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > > > <http://github.com/dongjinleekr>linkedin: kr.linkedin.com/in/
> > > > dongjinleekr
> > > > > <http://kr.linkedin.com/in/dongjinleekr>slideshare:
> > > > > www.slideshare.net/dongjinleekr
> > > > > <http://www.slideshare.net/dongjinleekr>*
> > > >
> > >
> > > --
> > > *Dongjin Lee*
> > >
> > > *A hitchhiker in the mathematical world.*
> > >
> > > *github:  <http://goog_969573159/>github.com/dongjinleekr
> > > <http://github.com/dongjinleekr>linkedin: kr.linkedin.com/in/
> > dongjinleekr
> > > <http://kr.linkedin.com/in/dongjinleekr>slideshare:
> www.slideshare.net/
> > dongjinleekr
> > > <http://www.slideshare.net/dongjinleekr>*
> > >
> >
>
> --
> *Dongjin Lee*
>
> *A hitchhiker in the mathematical world.*
>
> *github:  <http://goog_969573159/>github.com/dongjinleekr
> <http://github.com/dongjinleekr>linkedin: kr.linkedin.com/in/dongjinleekr
> <http://kr.linkedin.com/in/dongjinleekr>slideshare: 
> www.slideshare.net/dongjinleekr
> <http://www.slideshare.net/dongjinleekr>*
>

Reply via email to