Re: [VOTE] SPIP: An Official Kubernetes Operator for Apache Spark

2023-11-16 Thread Gabor Somogyi
+1 (non-binding) I think it's good from directional perspective. Apache Flink is already using this approach for quite some time in production. The overall conclusion is that it's a big gain :) G On Tue, Nov 14, 2023 at 6:42 PM L. C. Hsieh wrote: > Hi all, > > I’d like to start a vote for

Re: [DISCUSS] Flip the default value of Kafka offset fetching config (spark.sql.streaming.kafka.useDeprecatedOffsetFetching)

2022-10-13 Thread Gabor Somogyi
Hi Jungtaek, Good to hear that the new approach is working fine. +1 from my side. BR, G On Thu, Oct 13, 2022 at 4:12 AM Jungtaek Lim wrote: > Hi all, > > I would like to propose flipping the default value of Kafka offset > fetching config. The context is following: > > Before Spark 3.1,

Re: Issue on Spark on K8s with Proxy user on Kerberized HDFS : Spark-25355

2022-05-03 Thread Gabor Somogyi
where the wiki articles >>> are part of the troubleshooing process >>> https://issues.apache.org/jira/browse/HADOOP-7469 >>> >>> it's really disappointing when people escalate the problem to open >>> source developers before trying to fix the

Re: Issue on Spark on K8s with Proxy user on Kerberized HDFS : Spark-25355

2022-04-30 Thread Gabor Somogyi
Hi, Please be aware that ConnectionRefused exception is has nothing to do w/ authentication. See the description from Hadoop wiki: "You get a ConnectionRefused Exception when there is a machine at the address specified, but

Re: [DISCUSS] Migration guide on upgrading Kafka to 3.1 in Spark 3.3

2022-03-24 Thread Gabor Somogyi
I've had a small talk to the Kafka guys to find out a little bit more and the oversimplified conclusion is that if the producer version >= 3.0 and broker version < 0.11.0 with message format version V1 then either `enable.idempotence = false` needed or broker upgrade to 0.11.0+ is required to make

Re: [DISCUSS] Migration guide on upgrading Kafka to 3.1 in Spark 3.3

2022-03-18 Thread Gabor Somogyi
I've just read the related PR and seems like the situation is not so black and white as I've presumed purely from tech point of view... On Fri, 18 Mar 2022, 12:44 Gabor Somogyi, wrote: > Hi Jungtaek, > > I've taken a deeper look at the issue and here are my findings. > As far as I

Re: [DISCUSS] Migration guide on upgrading Kafka to 3.1 in Spark 3.3

2022-03-18 Thread Gabor Somogyi
Hi Jungtaek, I've taken a deeper look at the issue and here are my findings. As far as I'm concerned there are basically 2 ways with some minor decorations: * We care * We don't care I'm pretty sure users are clever enough but setting the expectation that all users are tracking Kafka KIPs

Re: [ANNOUNCE] Announcing Apache Spark 3.1.1

2021-03-03 Thread Gabor Somogyi
Good to hear and great work Hyukjin!  On Wed, 3 Mar 2021, 11:15 Jungtaek Lim, wrote: > Thanks Hyukjin for driving the huge release, and thanks everyone for > contributing the release! > > On Wed, Mar 3, 2021 at 6:54 PM angers zhu wrote: > >> Great work, Hyukjin ! >> >> Bests, >> Angers >> >>

Re: [VOTE] Release Spark 3.1.1 (RC3)

2021-02-24 Thread Gabor Somogyi
+1 (non-binding) Tested my added security related featues, found an issue but not a blocker. On Wed, 24 Feb 2021, 09:47 Hyukjin Kwon, wrote: > I remember HiveExternalCatalogVersionsSuite was flaky for a while which > is fixed in >

Re: [DISCUSS] Add RocksDB StateStore

2021-02-08 Thread Gabor Somogyi
+1 adding it any way. On Mon, 8 Feb 2021, 21:54 Holden Karau, wrote: > +1 for an external module. > > On Mon, Feb 8, 2021 at 11:51 AM Cheng Su wrote: > >> +1 for (2) adding to external module. >> >> I think this feature is useful and popular in practice, and option 2 is >> not conflict with

Re: [DISCUSS] Time to evaluate "continuous mode" in SS?

2020-09-15 Thread Gabor Somogyi
Hi Jungtaek, All I see at the moment is that most of the users choose Flink over Spark when continues processing is needed. Unless there is a revolution in this area there is no point to keep maintenance. 2.5 years is lot in bigdata industry. If there will be efforts in this area then happy to

Re: Apache Spark 3.1 Feature Expectation (Dec. 2020)

2020-07-01 Thread Gabor Somogyi
Hi Dongjoon, I would add JDBC Kerberos support w/ keytab: https://issues.apache.org/jira/browse/SPARK-12312 BR, G On Mon, Jun 29, 2020 at 6:07 PM Dongjoon Hyun wrote: > Hi, All. > > After a short celebration of Apache Spark 3.0, I'd like to ask you the > community opinion on Apache Spark 3.1

Re: Adding JIRA ID as the prefix for the test case name

2019-11-12 Thread Gabor Somogyi
+1 for having that consistent rule in test names. +1 for making it a guideline. +1 defining exact guides in general. Until now I've followed the alternative (only add the prefix when the JIRA's type is bug) and that way I knew that such tests contain edge cases. In case of new features I'm pretty

Re: FYI - filed bunch of issues for flaky tests in recent CI builds

2019-09-18 Thread Gabor Somogyi
Had a look at the Kafka test(SPARK-29136 ) and commented. BR, G On Wed, Sep 18, 2019 at 7:54 AM Jungtaek Lim wrote: > Hi devs, > > I've found bunch of test failures (intermittently) in both CI build for > master branch as well as PR builder

Re: Welcoming some new committers and PMC members

2019-09-10 Thread Gabor Somogyi
Congrats Guys! G On Tue, Sep 10, 2019 at 2:32 AM Matei Zaharia wrote: > Hi all, > > The Spark PMC recently voted to add several new committers and one PMC > member. Join me in welcoming them to their new roles! > > New PMC member: Dongjoon Hyun > > New committers: Ryan Blue, Liang-Chi Hsieh,

Re: [SS] KafkaSource doesn't use KafkaSourceInitialOffsetWriter for initial offsets?

2019-08-26 Thread Gabor Somogyi
OK, starting with this tomorrow... On Mon, 26 Aug 2019, 16:05 Jungtaek Lim, wrote: > Thanks! The patch is here: https://github.com/apache/spark/pull/25583 > > On Mon, Aug 26, 2019 at 11:02 PM Gabor Somogyi > wrote: > >> Just checked this and it's a copy-paste :) I

Re: [SS] KafkaSource doesn't use KafkaSourceInitialOffsetWriter for initial offsets?

2019-08-26 Thread Gabor Somogyi
Just checked this and it's a copy-paste :) It works properly when KafkaSourceInitialOffsetWriter used. Pull me in if review needed. BR, G On Mon, Aug 26, 2019 at 3:57 PM Jungtaek Lim wrote: > Nice finding! I don't see any reason to not use > KafkaSourceInitialOffsetWriter from KafkaSource, as

Re: JDBC connector for DataSourceV2

2019-07-15 Thread Gabor Somogyi
-25547>, and SPARK-28380 > <https://issues.apache.org/jira/browse/SPARK-28380> and finalize a plan. > > > > Xianyin > > > > *From: *Shiv Prashant Sood > *Date: *Sunday, July 14, 2019 at 2:59 AM > *To: *Gabor Somogyi > *Cc: *Xianyin Xin , Ryan Blue < >

Re: JDBC connector for DataSourceV2

2019-07-13 Thread Gabor Somogyi
Hi Guys, Don't know what's the intention exactly here but there is such a PR: https://github.com/apache/spark/pull/22560 If that's what we need maybe we can resurrect it. BTW, I'm also interested in... BR, G On Sat, Jul 13, 2019 at 4:09 AM Shiv Prashant Sood wrote: > Thanks all. I can also

Re: DSv1 removal

2019-06-21 Thread Gabor Somogyi
rs your question. If not, I'm happy to answer follow-ups > and we can add this as a topic in the next v2 sync on Wednesday. I'm also > planning on talking about metadata columns or function push-down from the > Kafka v2 PR at that sync, so you may want to attend. > > rb > > > O

DSv1 removal

2019-06-20 Thread Gabor Somogyi
Hi All, I've taken a look at the code and docs to find out when DSv1 sources has to be removed (in case of DSv2 replacement is implemented). After some digging I've found DSv1 sources which are already removed but in some cases v1 and v2 still exists in parallel. Can somebody please tell me

Re: Exposing JIRA issue types at GitHub PRs

2019-06-19 Thread Gabor Somogyi
ntributors skip adding the duplicated labels in new PR titles? > > > On Jun 17, 2019, at 4:21 PM, Gabor Somogyi > wrote: > > Dongjoon, I think it's useful. Thanks for adding it! > > On Mon, Jun 17, 2019 at 8:05 AM Dongjoon Hyun > wrote: > >> Thank you, Hyukjin !

Re: Exposing JIRA issue types at GitHub PRs

2019-06-17 Thread Gabor Somogyi
Dongjoon, I think it's useful. Thanks for adding it! On Mon, Jun 17, 2019 at 8:05 AM Dongjoon Hyun wrote: > Thank you, Hyukjin ! > > On Sun, Jun 16, 2019 at 4:12 PM Hyukjin Kwon wrote: > >> Labels look good and useful. >> >> On Sat, 15 Jun 2019, 02:36 Dongjoon Hyun, >> wrote: >> >>> Now, you

Re: dynamic allocation manager in SS

2019-05-27 Thread Gabor Somogyi
t; stavros.kontopou...@lightbend.com> wrote: > >> I am on k8s where there is no support yet afaik, there is wip wrt the >> shuffle service. So from your experience there are no issues with using the >> batch dynamic allocation version like there was before with dstreams as >> descr

Re: dynamic allocation manager in SS

2019-05-24 Thread Gabor Somogyi
st a thought. > > Στις Παρ, 24 Μαΐ 2019, 7:17 μ.μ. ο χρήστης Gabor Somogyi < > gabor.g.somo...@gmail.com> έγραψε: > >> Structured Streaming works differently. If no data arrives no tasks are >> executed (just had a case in this area). >> >> BR, >&g

Re: dynamic allocation manager in SS

2019-05-24 Thread Gabor Somogyi
Structured Streaming works differently. If no data arrives no tasks are executed (just had a case in this area). BR, G On Fri, 24 May 2019, 18:14 Stavros Kontopoulos, < stavros.kontopou...@lightbend.com> wrote: > Hi, > > Some while ago the streaming dynamic allocation part was added in

Re: What's the root cause of not supporting multiple aggregations in structured streaming?

2019-05-20 Thread Gabor Somogyi
There is PR for this but not yet merged. On Mon, May 20, 2019 at 10:13 AM 张万新 wrote: > Hi there, > > I'd like to know what's the root reason why multiple aggregations on > streaming dataframe is not allowed since it's a very useful feature, and > flink has supported it for a long time. > >

Jenkins issue

2019-03-22 Thread Gabor Somogyi
Hi All, Seems like there is a jenkins issue again. After a PR builder unit test failure I'm not able to open the jenkins page to take a look at the issue (it got stuck in infinite wait). BR, G

Re: Contribution

2019-02-12 Thread Gabor Somogyi
t; > вт, 12 февр. 2019 г. в 16:32, Gabor Somogyi : > >> Hi Valeria, >> >> Glad to hear you would like to contribute! It will be assigned to you >> when you create a PR. >> Before you create it please read the following guide which describe the >> details: h

Re: Contribution

2019-02-12 Thread Gabor Somogyi
Hi Valeria, Glad to hear you would like to contribute! It will be assigned to you when you create a PR. Before you create it please read the following guide which describe the details: https://spark.apache.org/contributing.html BR, G On Tue, Feb 12, 2019 at 2:28 PM Valeria Vasylieva <

Re: Welcome Jose Torres as a Spark committer

2019-01-30 Thread Gabor Somogyi
Congrats Jose! BR, G On Wed, Jan 30, 2019 at 9:05 AM Nuthan Reddy wrote: > Congrats Jose, > > Regards, > Nuthan Reddy > > > > On Wed, Jan 30, 2019 at 1:22 PM Marco Gaido > wrote: > >> Congrats, Jose! >> >> Bests, >> Marco >> >> Il giorno mer 30 gen 2019 alle ore 03:17 JackyLee ha >> scritto:

Re: [build system] speeding up maven build building only changed modules compared to master branch

2019-01-28 Thread Gabor Somogyi
Do you have some numbers how much is this faster? I'm asking it because previously I've evaluated another plugin and found the following: - Incremental build didn't bring too much even in bigger than spark projects - Incremental test was buggy and sometimes the required tests were not executed

DSv2 question

2019-01-24 Thread Gabor Somogyi
Hi All, Given org.apache.spark.sql.sources.v2.DataSourceOptions which states the following: * An immutable string-to-string map in which keys are case-insensitive. This is used to represent * data source options. Case-insensitivity can be reached many ways.The implementation provides lowercase

Re: Reading compacted Kafka topic is slow

2019-01-24 Thread Gabor Somogyi
Hi Tomas, Presume the 60 sec window means trigger interval. Maybe a quick win could be to try structured streaming because there the trigger interval is optional. If it is not specified, the system will check for availability of new data as soon as the previous processing has completed. BR, G

Re: Behavior of checkpointLocation from options vs setting conf spark.sql.streaming.checkpointLocation

2018-12-12 Thread Gabor Somogyi
Hi Shubham, I've just checked the latest master branch and I can confirm it works as you've described. As a workaround one can read the ** in the directory structure and can be set with .queryName("") before restart. BR, G On Tue, Dec 11, 2018 at 6:45 AM Shubham Chaurasia wrote: > Hi, > > I

Re: Implementation for exactly-once streaming sink

2018-12-06 Thread Gabor Somogyi
Hi Eric, In order to have exactly-once one need re-playable source and idempotent sink. The cases what you've mentioned are covering the 2 main group of issues. Practically any kind of programming problem can end-up in duplicated data (even in the code which feeds kafka). Don't know why have you

Re: [SS] No reponse on a PR: Report numOutputRows in SinkProgress

2018-11-30 Thread Gabor Somogyi
Hi Vaclav, As I see it's conflicting at the moment. BR, G On Fri, Nov 30, 2018 at 11:49 AM Vaclav Kosar wrote: > Fellow Structured Streamers, > > I am having trouble getting any feedback on my PR for reporting number > of written rows in Structured Streaming: >

Re: SPIP: Support Kafka delegation token in Structured Streaming

2018-10-01 Thread Gabor Somogyi
gt; Kafka as a built-in support. > > BTW I think there's no much difference in support structured streaming and > DStream, maybe we can set both as goal. > > Thanks > Saisai > > Gabor Somogyi 于2018年9月27日周四 下午7:58写道: > >> Hi all, >> >> I am writing this e

Re: SPIP: Support Kafka delegation token in Structured Streaming

2018-10-01 Thread Gabor Somogyi
tion > token for HDFS and HBase. I'll try to review the doc in general, and try to > learn it, and review again based on understanding. > > Thanks, > Jungtaek Lim (HeartSaVioR) > > 2018년 9월 27일 (목) 오후 8:58, Gabor Somogyi 님이 작성: > >> Hi all, >> >> I am writin

SPIP: Support Kafka delegation token in Structured Streaming

2018-09-27 Thread Gabor Somogyi
Hi all, I am writing this e-mail in order to discuss the delegation token support for kafka feature which is reported in SPARK-25501 . I've prepared a SPIP