Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-26848][SQL][SS] Introduce new option to Kafka source: offset by timestamp (starting/ending) #23747

Closed
wants to merge 3 commits into from

Conversation

HeartSaVioR
Copy link
Contributor

@HeartSaVioR HeartSaVioR commented Feb 8, 2019

What changes were proposed in this pull request?

This patch introduces new options "startingOffsetsByTimestamp" and "endingOffsetsByTimestamp" to set specific timestamp per topic (since we're unlikely to set the different value per partition) to let source starts reading from offsets which have equal of greater timestamp, and ends reading until offsets which have equal of greater timestamp.

The new option would be optional of course, and take preference over existing offset options.

How was this patch tested?

New unit tests added. Also manually tested basic functionality with Kafka 2.0.0 server.

Running query below

val df = spark.read.format("kafka")
  .option("kafka.bootstrap.servers", "localhost:9092")
  .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1")
  .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""")
  .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""")
  .load().selectExpr("CAST(value AS STRING)")

df.show()

with below records (one string which number part remarks when they're put after such timestamp) in

topic spark_26848_test_v1

hello1 1549669142193
world1 1549669142193
hellow1 1549669240965
world1 1549669240965
hello1 1549669265676
world1 1549669265676

topic spark_26848_test_2_v1

hello2 1549669142193
world2 1549669142193
hello2 1549669240965
world2 1549669240965
hello2 1549669265676
world2 1549669265676

the result of df.show() follows:

+--------------------+
|               value|
+--------------------+
|world1 1549669240965|
|world1 1549669142193|
|world2 1549669240965|
|hello2 1549669240965|
|hellow1 154966924...|
|hello2 1549669265676|
|hello1 1549669142193|
|world2 1549669265676|
+--------------------+

Note that endingOffsets (as well as endingOffsetsByTimestamp) are exclusive.

@SparkQA
Copy link

SparkQA commented Feb 8, 2019

Test build #102099 has finished for PR 23747 at commit ed94115.

  • This patch fails Scala style tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HeartSaVioR
Copy link
Contributor Author

@SparkQA
Copy link

SparkQA commented Feb 8, 2019

Test build #102100 has finished for PR 23747 at commit 3479e87.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Feb 8, 2019

Test build #102103 has finished for PR 23747 at commit 49b2ff0.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HeartSaVioR HeartSaVioR changed the title [WIP][SPARK-26848][SQL] Introduce new option to Kafka source: offset by timestamp (starting/ending) [SPARK-26848][SQL] Introduce new option to Kafka source: offset by timestamp (starting/ending) Feb 8, 2019
@zsxwing
Copy link
Member

zsxwing commented Feb 11, 2019

If a user uses a Kafka cluster which runs using an old version that doesn't support timestamp APIs, will their query fail?

@HeartSaVioR
Copy link
Contributor Author

If a user uses a Kafka cluster which runs using an old version that doesn't support timestamp APIs, will their query fail?

We can just give it a try with Kafka 0.10.0. (I guess Kafka 0.9.x is not API compatible so we don't need to care about it.) Let me install that version and test and get back to you.

@HeartSaVioR
Copy link
Contributor Author

HeartSaVioR commented Feb 11, 2019

I realized I missed pushing commit adding required version of Kafka to use this feature. Just pushed.

Btw, I'm curious we would mind the situation pretty seriously that end users will use Spark 3.0.0 against Kafka 0.10.0.x (last bugfix released to Aug 2016). When end users put Spark 3.0.0 in their production the fastest would be mid this year, then we're supposing end users to use Kafka released 3 years ago while they're brave to be early adopters of newer version of Spark - feel a bit far from realistic.

@HeartSaVioR
Copy link
Contributor Author

HeartSaVioR commented Feb 11, 2019

Just ran the query with Kafka 0.10.0.

batch query

val df = spark.read.format("kafka")
  .option("kafka.bootstrap.servers", "localhost:9092")
  .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1")
  .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""")
  .option("endingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669265676, "spark_26848_test_2_v1": 1549699265676}""")
  .load().selectExpr("CAST(value AS STRING)")

df.show()

result:

19/02/12 06:01:47 WARN KafkaOffsetReader: Error in attempt 1 getting Kafka offsets:
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].
19/02/12 06:01:48 WARN KafkaOffsetReader: Error in attempt 2 getting Kafka offsets:
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].
19/02/12 06:01:49 WARN KafkaOffsetReader: Error in attempt 3 getting Kafka offsets:
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].
org.apache.spark.SparkException: Exception thrown in awaitResult:
  at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:226)
  at org.apache.spark.sql.kafka010.KafkaOffsetReader.runUninterruptibly(KafkaOffsetReader.scala:375)
  at org.apache.spark.sql.kafka010.KafkaOffsetReader.fetchSpecificTimestampBasedOffsets(KafkaOffsetReader.scala:184)
  at org.apache.spark.sql.kafka010.KafkaRelation.getPartitionOffsets(KafkaRelation.scala:147)
  at org.apache.spark.sql.kafka010.KafkaRelation.buildScan(KafkaRelation.scala:72)
...
Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].

SS query

val df = spark.readStream.format("kafka")
  .option("kafka.bootstrap.servers", "localhost:9092")
  .option("subscribe", "spark_26848_test_v1,spark_26848_test_2_v1")
  .option("startingOffsetsByTimestamp", """{"spark_26848_test_v1": 1549669142193, "spark_26848_test_2_v1": 1549669240965}""")
  .load().selectExpr("CAST(value AS STRING)")

val query = df.writeStream
  .format("console")  // <-- use ConsoleSink
  .option("truncate", false)
  .option("numRows", 10)
  .queryName("rate-console")
  .start()

query.awaitTermination()

result:

19/02/12 06:09:10 WARN KafkaOffsetReader: Error in attempt 1 getting Kafka offsets:
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].
19/02/12 06:09:11 WARN KafkaOffsetReader: Error in attempt 2 getting Kafka offsets:
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].
19/02/12 06:09:12 WARN KafkaOffsetReader: Error in attempt 3 getting Kafka offsets:
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].
19/02/12 06:09:13 ERROR MicroBatchExecution: Query rate-console [id = 00429123-93dd-4c65-8c74-61aa8eb11f82, runId = c0263eb7-b31f-4ba6-9bd6-d076e89e5803] terminated with error
org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].
org.apache.spark.sql.streaming.StreamingQueryException: Query rate-console [id = 00429123-93dd-4c65-8c74-61aa8eb11f82, runId = c0263eb7-b31f-4ba6-9bd6-d076e89e5803] terminated with exception: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].
  at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:307)
  at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:198)
Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: The broker does not support LIST_OFFSETS with version in range [1,4]. The supported range is [0,0].

@SparkQA
Copy link

SparkQA commented Feb 11, 2019

Test build #102210 has finished for PR 23747 at commit ca0e548.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HeartSaVioR
Copy link
Contributor Author

HeartSaVioR commented Feb 13, 2019

Does someone involved in Kafka community?

I'm seeking Kafka doc for offsetsForTimes, but unlike we update doc as well when introducing a new feature, origin PR for offsetsForTimes doesn't seem to touch doc. (Not sure whether they have doc outside of main repo.)

https://github.com/apache/kafka/pull/1215/files

Does it expect to refer/read KIP page directly in this case?

@koeninger
Copy link
Contributor

koeninger commented Feb 13, 2019 via email

@gaborgsomogyi
Copy link
Contributor

That's also a good suggestion.
I meant something like: timestamp is dependent on log.message.timestamp.type for further details please see https://kafka.apache.org/documentation/...
That said either is fine for me.

@koeninger
Copy link
Contributor

koeninger commented Feb 13, 2019 via email

@HeartSaVioR
Copy link
Contributor Author

OK. Got it. Thanks both of you for the nice recommendation.

@SparkQA
Copy link

SparkQA commented Feb 13, 2019

Test build #102312 has finished for PR 23747 at commit 64ceaec.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HeartSaVioR
Copy link
Contributor Author

HeartSaVioR commented Mar 7, 2019

IMHO SPARK-23539 and this (SPARK-26848) are good options to be included for Spark 3.0, given other streaming frameworks support these and the features don't break existing thing. This option would also help on batch queries to restrict the range by timestamp which is intuitive than raw offsets.

@@ -29,4 +29,13 @@ trait KafkaTest extends BeforeAndAfterAll {
super.afterAll()
CachedKafkaProducer.clear()
}

def waitForBiggerTimestamp(curTimestamp: Long): Long = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

AFAIK timestamp field can be overridden in the ProducerRecord, can't be? Looks like this part generates a lot of unit test wait.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah yes nice suggestion. Will address.

@SparkQA
Copy link

SparkQA commented Mar 7, 2019

Test build #103140 has finished for PR 23747 at commit 8d0e158.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@gaborgsomogyi
Copy link
Contributor

Now I have a cluster and started to evaluate this in-depth, it may take some time...

@HeartSaVioR
Copy link
Contributor Author

@tdas @zsxwing @jose-torres Could we consider start reviewing this one to add this feature in 3.0.0?

@jose-torres
Copy link
Contributor

@gaborgsomogyi, what is it that you're evaluating in depth?

In general, this seems like a reasonable change. I'd make the docs more up-front about the fact that it's a pass-through feature: Spark doesn't do any interpretation or reasoning about the timestamp, it just passes it directly to Kafka for Kafka to translate into an offset. (Otherwise I worry people will get confused and think Spark is responsible for some translation problem they're facing.)

@@ -179,6 +179,56 @@ private[kafka010] class KafkaOffsetReader(
KafkaSourceOffset(fetched)
}

def fetchSpecificTimestampBasedOffsets(topicTimestamps: Map[String, Long]): KafkaSourceOffset = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there a way to share more code with fetchSpecificOffsets() here? I know a lot of the implementation is a bit different, but I don't like the idea of having complicated things like "workaround for KAFKA-7703" in multiple independent places.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'll try to do it: if it turned out hard to achieve I'll leave a comment here.

This option leverages `KafkaConsumer.offsetForTimes`: please refer https://kafka.apache.org/21/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#offsetsForTimes-java.util.Map- for details.<p/>
Also the meaning of `timestamp` here can be vary according to Kafka configuration (`log.message.timestamp.type`): please refer https://kafka.apache.org/documentation/ for further details.<p/>
Note: This option requires Kafka 0.10.1.0 or higher.<p/>
Note2: `startingOffsetsByTimestamp` takes precedence over `startingOffsets`.<p/>
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wish there were a way to combine the options instead of having precedence logic, but I can't think of one.

Copy link
Contributor Author

@HeartSaVioR HeartSaVioR Mar 16, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah... it might be possible if we use timestamp as some of custom format like ts:<timestamp>, and try to parse it as long for offset, and fail-back to custom format for timestamp. I already considered like this but didn't feel good.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Agreed.

val assignedTopics = partitions.asScala.map(_.topic())
assert(assignedTopics == topicTimestamps.keySet,
"If starting/endingOffsetsByTimestamp contains specific offsets, you must specify all " +
s"topics. Specified: ${topicTimestamps.keySet} Assigned: $assignedTopics")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should test that 0 and System.currentTimeMillis() work straightforwardly as timestamp offsets, so people can use them the same way as -2 and -1 in the non-timestamp case.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Given we want to state that Spark transparently passes timestamp to Kafka without interpreting and reasoning, would we need to add some test to reason about timestamp semantics?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm never opposed to more tests :)

@gaborgsomogyi
Copy link
Contributor

@jose-torres I also think its reasonable change and that's the reason why I'll test on cluster with corner cases, etc... but only next Monday because I'm on vacation...

consumer.poll(0)
val partitions = consumer.assignment()

// Call `position` to wait until the potential offset request triggered by `poll(0)` is
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is actually not executed for fetchEarliestOffsets right now, but I wouldn't feel it hurts much.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Never mind just added a flag to handle it.

@HeartSaVioR
Copy link
Contributor Author

Due to deal with function parameters with long param names, the indentation may look a bit weird. Honestly I'm not sure I can point out where the indentations are broken. Comments regarding indentation welcome!

@SparkQA
Copy link

SparkQA commented Mar 16, 2019

Test build #103571 has finished for PR 23747 at commit 7b51425.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HeartSaVioR
Copy link
Contributor Author

HeartSaVioR commented Sep 16, 2019

I don't feel that qualified to review this, but see others have generally approved.

I see. No problem and thanks for reviewing even the patch is not familiar for you. I can wait for other reviewers who can decide to merge.

Is there any impact to users who do not specify these new properties? does it overlap with or duplicate any existing "offset" functionality? Those would be my key review questions.

No. It provides another way to set "offset", by timestamp.

For now, end users need to set exact offset number or either have to set latest/earliest, and when they want to run the query starting from specific time point they need to know about exact offset which is inserted at that time. While end users may retrieve it from cli tool (not 100% sure but given they expose API...), it's not convenient to retrieve the offset from Kafka for the time point and set to Spark option. There's another benefit for this change - once they specify the offset to Spark option, unless they also leave comment to describe where the offset came from, the offset number is not showing the intention that they want to run from specific time point. After the patch the intention could be represented very clear on their Spark app.

Regarding Kafka 0.10 support, yes I think it could be reasonable to drop support for < 1.0. ... Would there be any significant upside for Spark, like simplifying code or assumptions, making it easier to support, taking advantage of newer features?

Maybe we don't need to guide about version issue for both this (>= 0.10) and Kafka header support (>= 0.11). We already use pretty high version of Kafka client so there's no significant change (benefits on code side) on drop supporting old versions.

@SparkQA
Copy link

SparkQA commented Sep 16, 2019

Test build #110666 has finished for PR 23747 at commit 22b7dba.

  • This patch fails to build.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HeartSaVioR
Copy link
Contributor Author

HeartSaVioR commented Sep 16, 2019

This now breaks part of change on 88c8d5e. Need to rebase and fix.

…mestamp (starting/ending)

* Remove unnecessary code, and fix scalastyle
* Documentation
* Add note on Kafka version to support this feature
* Address review comment from gaborgsomogyi: added more information regarding timestamp on the doc
* Address review comment from gaborgsomogyi
* Address review comments from jose-torres
* Minor change: fetchEarliestOffset doesn't call unnecessary 'position'
* Address review comments from gaborgsomogyi
* Address review comment from gaborgsomogyi
* Replace ` with <code> tag
* Support per-partition timestamp
* Some cleaning up
* Make query fail when there's no matched offset for starting offset timestamp
@SparkQA
Copy link

SparkQA commented Sep 16, 2019

Test build #110667 has finished for PR 23747 at commit c8a4938.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Sep 19, 2019

Test build #4877 has finished for PR 23747 at commit c8a4938.

  • This patch passes all tests.
  • This patch does not merge cleanly.
  • This patch adds no public classes.

@HeartSaVioR
Copy link
Contributor Author

retest this, please

@SparkQA
Copy link

SparkQA commented Sep 20, 2019

Test build #111023 has finished for PR 23747 at commit c8a4938.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@srowen
Copy link
Member

srowen commented Sep 23, 2019

@HeartSaVioR this is good to go? following #23747 (comment)

@HeartSaVioR
Copy link
Contributor Author

@srowen
Yes, c8a4938 is the commit to fix compilation failure due to 88c8d5e.

@srowen
Copy link
Member

srowen commented Sep 23, 2019

@gaborgsomogyi @koeninger any more thoughts on this one or looking OK?

@gaborgsomogyi
Copy link
Contributor

@srowen no further comment, after revisit my approval still stands.

Regarding Kafka 0.10 support, yes I think it could be reasonable to drop support for < 1.0. ... Would there be any significant upside for Spark, like simplifying code or assumptions, making it easier to support, taking advantage of newer features?

Since the actual connector is 0.10+ compatible it would be additional effort to drop < 1.0 support so I don't see benefit here.

@koeninger
Copy link
Contributor

It's a good idea.
I gave it another read today, looks OK to me.

@srowen
Copy link
Member

srowen commented Sep 24, 2019

Merged to master

@srowen srowen closed this in 4513f1c Sep 24, 2019
@HeartSaVioR
Copy link
Contributor Author

Thanks all for reviewing and merging!

@HeartSaVioR HeartSaVioR deleted the SPARK-26848 branch September 24, 2019 00:29
cloud-fan pushed a commit that referenced this pull request Jan 6, 2020
…-integration page

### What changes were proposed in this pull request?
Fix the disorder of `structured-streaming-kafka-integration` page caused by #23747.

### Why are the changes needed?
A typo messed up the HTML page.

### Does this PR introduce any user-facing change?
No

### How was this patch tested?
Locally test by Jekyll.
Before:
![image](https://user-images.githubusercontent.com/4833765/71793803-6c0a1e80-3079-11ea-8fce-f0f94fd6929c.png)
After:
![image](https://user-images.githubusercontent.com/4833765/71793807-72989600-3079-11ea-9e12-f83437eeb7c0.png)

Closes #27098 from xuanyuanking/SPARK-30426.

Authored-by: Yuanjian Li <xyliyuanjian@gmail.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
HeartSaVioR added a commit that referenced this pull request May 25, 2021
…bing topic-partitions in Kafka source

### What changes were proposed in this pull request?

This patch is a follow-up of SPARK-26848 (#23747). In SPARK-26848, we decided to open possibility to let end users set individual timestamp per partition. But in many cases, specifying timestamp represents the intention that we would want to go back to specific timestamp and reprocess records, which should be applied to all topics and partitions.

This patch proposes to provide a way to set a global timestamp across topic-partitions which the source is subscribing to, so that end users can set all offsets by specific timestamp easily. To provide the way to config the timestamp easier, the new options only receive "a" timestamp for start/end timestamp.

New options introduced in this PR:

* startingTimestamp
* endingTimestamp

All two options receive timestamp as string.

There're priorities for options regarding starting/ending offset as we will have three options for start offsets and another three options for end offsets. Priorities are following:

* starting offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets
* ending offsets: startingTimestamp -> startingOffsetsByTimestamp -> startingOffsets

### Why are the changes needed?

Existing option to specify timestamp as offset is quite verbose if there're a lot of partitions across topics. Suppose there're 100s of partitions in a topic, the json should contain 100s of times of the same timestamp.

Also, the number of partitions can also change, which requires either:

* fixing the code if the json is statically created
* introducing the dependencies on Kafka client and deal with Kafka API on crafting json programmatically

Both approaches are even not "acceptable" if we're dealing with ad-hoc query; anyone doesn't want to write the code more complicated than the query itself. Flink [provides the option](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/connectors/datastream/kafka/#kafka-consumers-start-position-configuration) to specify a timestamp for all topic-partitions like this PR, and even doesn't provide the option to specify the timestamp per topic-partition.

With this PR, end users are only required to provide a single timestamp value. No more complicated JSON format end users need to know about the structure.

### Does this PR introduce _any_ user-facing change?

Yes, this PR introduces two new options, described in above section.

Doc changes are following:

![스크린샷 2021-05-21 오후 12 01 02](https://user-images.githubusercontent.com/1317309/119076244-3034e680-ba2d-11eb-8323-0e227932d2e5.png)
![스크린샷 2021-05-21 오후 12 01 12](https://user-images.githubusercontent.com/1317309/119076255-35923100-ba2d-11eb-9d79-538a7f9ee738.png)
![스크린샷 2021-05-21 오후 12 01 24](https://user-images.githubusercontent.com/1317309/119076264-39be4e80-ba2d-11eb-8265-ac158f55c360.png)
![스크린샷 2021-05-21 오후 12 06 01](https://user-images.githubusercontent.com/1317309/119076271-3d51d580-ba2d-11eb-98ea-35fd72b1bbfc.png)

### How was this patch tested?

New UTs covering new functionalities. Also manually tested via simple batch & streaming queries.

Closes #32609 from HeartSaVioR/SPARK-29223-v2.

Authored-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
8 participants