You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@spark.apache.org by Wenchen Fan <cl...@gmail.com> on 2020/02/12 15:42:43 UTC

[DISCUSS] naming policy of Spark configs

Hi all,

I'd like to discuss the naming policy of Spark configs, as for now it
depends on personal preference which leads to inconsistent namings.

In general, the config name should be a noun that describes its meaning
clearly.
Good examples:
spark.sql.session.timeZone
spark.sql.streaming.continuous.executorQueueSize
spark.sql.statistics.histogram.numBins
Bad examples:
spark.sql.defaultSizeInBytes (default size for what?)

Also note that, config name has many parts, joined by dots. Each part is a
namespace. Don't create namespace unnecessarily.
Good example:
spark.sql.execution.rangeExchange.sampleSizePerPartition
spark.sql.execution.arrow.maxRecordsPerBatch
Bad examples:
spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a useful
namespace, better to be .buffer.inMemoryThreshold)

For a big feature, usually we need to create an umbrella config to turn it
on/off, and other configs for fine-grained controls. These configs should
share the same namespace, and the umbrella config should be named like
featureName.enabled. For example:
spark.sql.cbo.enabled
spark.sql.cbo.starSchemaDetection
spark.sql.cbo.starJoinFTRatio
spark.sql.cbo.joinReorder.enabled
spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good namespace)
spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good namespace)

For boolean configs, in general it should end with a verb, e.g.
spark.sql.join.preferSortMergeJoin. If the config is for a feature and you
can't find a good verb for the feature, featureName.enabled is also good.

I'll update https://spark.apache.org/contributing.html after we reach a
consensus here. Any comments are welcome!

Thanks,
Wenchen

RE: [DISCUSS] naming policy of Spark configs

Posted by Kazuaki Ishizaki <IS...@jp.ibm.com>.
+1 if we add them to Alternative config.

Kazuaki Ishizaki



From:   Takeshi Yamamuro <li...@gmail.com>
To:     Wenchen Fan <cl...@gmail.com>
Cc:     Spark dev list <de...@spark.apache.org>
Date:   2020/02/13 16:02
Subject:        [EXTERNAL] Re: [DISCUSS] naming policy of Spark configs



+1; the idea sounds reasonable.

Bests,
Takeshi

On Thu, Feb 13, 2020 at 12:39 PM Wenchen Fan <cl...@gmail.com> wrote:
Hi Dongjoon,

It's too much work to revisit all the configs that added in 3.0, but I'll 
revisit the recent commits that update config names and see if they follow 
the new policy.


Hi Reynold,

There are a few interval configs:
spark.sql.streaming.fileSink.log.compactInterval
spark.sql.streaming.continuous.executorPollIntervalMs

I think it's better to put the interval unit in the config name, like 
`executorPollIntervalMs`. Also the config should be created with 
`.timeConf`, so that users can set values like "1 second", "2 minutes", 
etc.

There is no config that uses date/timestamp as value AFAIK.


Thanks,
Wenchen

On Thu, Feb 13, 2020 at 11:29 AM Jungtaek Lim <
kabhwan.opensource@gmail.com> wrote:
+1 Thanks for the proposal. Looks very reasonable to me.

On Thu, Feb 13, 2020 at 10:53 AM Hyukjin Kwon <gu...@gmail.com> wrote:
+1.

2020년 2월 13일 (목) 오전 9:30, Gengliang Wang <
gengliang.wang@databricks.com>님이 작성:
+1, this is really helpful. We should make the SQL configurations 
consistent and more readable.

On Wed, Feb 12, 2020 at 3:33 PM Rubén Berenguel <rb...@gmail.com> 
wrote:
I love it, it will make configs easier to read and write. Thanks Wenchen. 

R

On 13 Feb 2020, at 00:15, Dongjoon Hyun <do...@gmail.com> wrote:


Thank you, Wenchen.

The new policy looks clear to me. +1 for the explicit policy.

So, are we going to revise the existing conf names before 3.0.0 release?

Or, is it applied to new up-coming configurations from now?

Bests,
Dongjoon.

On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan <cl...@gmail.com> wrote:
Hi all,

I'd like to discuss the naming policy of Spark configs, as for now it 
depends on personal preference which leads to inconsistent namings.

In general, the config name should be a noun that describes its meaning 
clearly.
Good examples:
spark.sql.session.timeZone
spark.sql.streaming.continuous.executorQueueSize
spark.sql.statistics.histogram.numBins
Bad examples:
spark.sql.defaultSizeInBytes (default size for what?)

Also note that, config name has many parts, joined by dots. Each part is a 
namespace. Don't create namespace unnecessarily.
Good example:
spark.sql.execution.rangeExchange.sampleSizePerPartition
spark.sql.execution.arrow.maxRecordsPerBatch
Bad examples:
spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a useful 
namespace, better to be .buffer.inMemoryThreshold)

For a big feature, usually we need to create an umbrella config to turn it 
on/off, and other configs for fine-grained controls. These configs should 
share the same namespace, and the umbrella config should be named like 
featureName.enabled. For example:
spark.sql.cbo.enabled
spark.sql.cbo.starSchemaDetection
spark.sql.cbo.starJoinFTRatio
spark.sql.cbo.joinReorder.enabled
spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good namespace
)
spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good namespace
)

For boolean configs, in general it should end with a verb, e.g. 
spark.sql.join.preferSortMergeJoin. If the config is for a feature and you 
can't find a good verb for the feature, featureName.enabled is also good.

I'll update https://spark.apache.org/contributing.html after we reach a 
consensus here. Any comments are welcome!

Thanks,
Wenchen




-- 
---
Takeshi Yamamuro



Re: [DISCUSS] naming policy of Spark configs

Posted by Takeshi Yamamuro <li...@gmail.com>.
+1; the idea sounds reasonable.

Bests,
Takeshi

On Thu, Feb 13, 2020 at 12:39 PM Wenchen Fan <cl...@gmail.com> wrote:

> Hi Dongjoon,
>
> It's too much work to revisit all the configs that added in 3.0, but I'll
> revisit the recent commits that update config names and see if they follow
> the new policy.
>
>
> Hi Reynold,
>
> There are a few interval configs:
> spark.sql.streaming.fileSink.log.compactInterval
> spark.sql.streaming.continuous.executorPollIntervalMs
>
> I think it's better to put the interval unit in the config name, like
> `executorPollIntervalMs`. Also the config should be created with
> `.timeConf`, so that users can set values like "1 second", "2 minutes", etc.
>
> There is no config that uses date/timestamp as value AFAIK.
>
>
> Thanks,
> Wenchen
>
> On Thu, Feb 13, 2020 at 11:29 AM Jungtaek Lim <
> kabhwan.opensource@gmail.com> wrote:
>
>> +1 Thanks for the proposal. Looks very reasonable to me.
>>
>> On Thu, Feb 13, 2020 at 10:53 AM Hyukjin Kwon <gu...@gmail.com>
>> wrote:
>>
>>> +1.
>>>
>>> 2020년 2월 13일 (목) 오전 9:30, Gengliang Wang <ge...@databricks.com>님이
>>> 작성:
>>>
>>>> +1, this is really helpful. We should make the SQL configurations
>>>> consistent and more readable.
>>>>
>>>> On Wed, Feb 12, 2020 at 3:33 PM Rubén Berenguel <rb...@gmail.com>
>>>> wrote:
>>>>
>>>>> I love it, it will make configs easier to read and write. Thanks
>>>>> Wenchen.
>>>>>
>>>>> R
>>>>>
>>>>> On 13 Feb 2020, at 00:15, Dongjoon Hyun <do...@gmail.com>
>>>>> wrote:
>>>>>
>>>>> 
>>>>> Thank you, Wenchen.
>>>>>
>>>>> The new policy looks clear to me. +1 for the explicit policy.
>>>>>
>>>>> So, are we going to revise the existing conf names before 3.0.0
>>>>> release?
>>>>>
>>>>> Or, is it applied to new up-coming configurations from now?
>>>>>
>>>>> Bests,
>>>>> Dongjoon.
>>>>>
>>>>> On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan <cl...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Hi all,
>>>>>>
>>>>>> I'd like to discuss the naming policy of Spark configs, as for now it
>>>>>> depends on personal preference which leads to inconsistent namings.
>>>>>>
>>>>>> In general, the config name should be a noun that describes its
>>>>>> meaning clearly.
>>>>>> Good examples:
>>>>>> spark.sql.session.timeZone
>>>>>> spark.sql.streaming.continuous.executorQueueSize
>>>>>> spark.sql.statistics.histogram.numBins
>>>>>> Bad examples:
>>>>>> spark.sql.defaultSizeInBytes (default size for what?)
>>>>>>
>>>>>> Also note that, config name has many parts, joined by dots. Each part
>>>>>> is a namespace. Don't create namespace unnecessarily.
>>>>>> Good example:
>>>>>> spark.sql.execution.rangeExchange.sampleSizePerPartition
>>>>>> spark.sql.execution.arrow.maxRecordsPerBatch
>>>>>> Bad examples:
>>>>>> spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a
>>>>>> useful namespace, better to be .buffer.inMemoryThreshold)
>>>>>>
>>>>>> For a big feature, usually we need to create an umbrella config to
>>>>>> turn it on/off, and other configs for fine-grained controls. These configs
>>>>>> should share the same namespace, and the umbrella config should be named
>>>>>> like featureName.enabled. For example:
>>>>>> spark.sql.cbo.enabled
>>>>>> spark.sql.cbo.starSchemaDetection
>>>>>> spark.sql.cbo.starJoinFTRatio
>>>>>> spark.sql.cbo.joinReorder.enabled
>>>>>> spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good
>>>>>> namespace)
>>>>>> spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good
>>>>>> namespace)
>>>>>>
>>>>>> For boolean configs, in general it should end with a verb, e.g.
>>>>>> spark.sql.join.preferSortMergeJoin. If the config is for a feature
>>>>>> and you can't find a good verb for the feature, featureName.enabled
>>>>>> is also good.
>>>>>>
>>>>>> I'll update https://spark.apache.org/contributing.html after we
>>>>>> reach a consensus here. Any comments are welcome!
>>>>>>
>>>>>> Thanks,
>>>>>> Wenchen
>>>>>>
>>>>>>
>>>>>>

-- 
---
Takeshi Yamamuro

Re: [DISCUSS] naming policy of Spark configs

Posted by Wenchen Fan <cl...@gmail.com>.
Hi Dongjoon,

It's too much work to revisit all the configs that added in 3.0, but I'll
revisit the recent commits that update config names and see if they follow
the new policy.


Hi Reynold,

There are a few interval configs:
spark.sql.streaming.fileSink.log.compactInterval
spark.sql.streaming.continuous.executorPollIntervalMs

I think it's better to put the interval unit in the config name, like
`executorPollIntervalMs`. Also the config should be created with
`.timeConf`, so that users can set values like "1 second", "2 minutes", etc.

There is no config that uses date/timestamp as value AFAIK.


Thanks,
Wenchen

On Thu, Feb 13, 2020 at 11:29 AM Jungtaek Lim <ka...@gmail.com>
wrote:

> +1 Thanks for the proposal. Looks very reasonable to me.
>
> On Thu, Feb 13, 2020 at 10:53 AM Hyukjin Kwon <gu...@gmail.com> wrote:
>
>> +1.
>>
>> 2020년 2월 13일 (목) 오전 9:30, Gengliang Wang <ge...@databricks.com>님이
>> 작성:
>>
>>> +1, this is really helpful. We should make the SQL configurations
>>> consistent and more readable.
>>>
>>> On Wed, Feb 12, 2020 at 3:33 PM Rubén Berenguel <rb...@gmail.com>
>>> wrote:
>>>
>>>> I love it, it will make configs easier to read and write. Thanks
>>>> Wenchen.
>>>>
>>>> R
>>>>
>>>> On 13 Feb 2020, at 00:15, Dongjoon Hyun <do...@gmail.com>
>>>> wrote:
>>>>
>>>> 
>>>> Thank you, Wenchen.
>>>>
>>>> The new policy looks clear to me. +1 for the explicit policy.
>>>>
>>>> So, are we going to revise the existing conf names before 3.0.0 release?
>>>>
>>>> Or, is it applied to new up-coming configurations from now?
>>>>
>>>> Bests,
>>>> Dongjoon.
>>>>
>>>> On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan <cl...@gmail.com>
>>>> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> I'd like to discuss the naming policy of Spark configs, as for now it
>>>>> depends on personal preference which leads to inconsistent namings.
>>>>>
>>>>> In general, the config name should be a noun that describes its
>>>>> meaning clearly.
>>>>> Good examples:
>>>>> spark.sql.session.timeZone
>>>>> spark.sql.streaming.continuous.executorQueueSize
>>>>> spark.sql.statistics.histogram.numBins
>>>>> Bad examples:
>>>>> spark.sql.defaultSizeInBytes (default size for what?)
>>>>>
>>>>> Also note that, config name has many parts, joined by dots. Each part
>>>>> is a namespace. Don't create namespace unnecessarily.
>>>>> Good example:
>>>>> spark.sql.execution.rangeExchange.sampleSizePerPartition
>>>>> spark.sql.execution.arrow.maxRecordsPerBatch
>>>>> Bad examples:
>>>>> spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a useful
>>>>> namespace, better to be .buffer.inMemoryThreshold)
>>>>>
>>>>> For a big feature, usually we need to create an umbrella config to
>>>>> turn it on/off, and other configs for fine-grained controls. These configs
>>>>> should share the same namespace, and the umbrella config should be named
>>>>> like featureName.enabled. For example:
>>>>> spark.sql.cbo.enabled
>>>>> spark.sql.cbo.starSchemaDetection
>>>>> spark.sql.cbo.starJoinFTRatio
>>>>> spark.sql.cbo.joinReorder.enabled
>>>>> spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good
>>>>> namespace)
>>>>> spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good
>>>>> namespace)
>>>>>
>>>>> For boolean configs, in general it should end with a verb, e.g.
>>>>> spark.sql.join.preferSortMergeJoin. If the config is for a feature
>>>>> and you can't find a good verb for the feature, featureName.enabled
>>>>> is also good.
>>>>>
>>>>> I'll update https://spark.apache.org/contributing.html after we reach
>>>>> a consensus here. Any comments are welcome!
>>>>>
>>>>> Thanks,
>>>>> Wenchen
>>>>>
>>>>>
>>>>>

Re: [DISCUSS] naming policy of Spark configs

Posted by Jungtaek Lim <ka...@gmail.com>.
+1 Thanks for the proposal. Looks very reasonable to me.

On Thu, Feb 13, 2020 at 10:53 AM Hyukjin Kwon <gu...@gmail.com> wrote:

> +1.
>
> 2020년 2월 13일 (목) 오전 9:30, Gengliang Wang <ge...@databricks.com>님이
> 작성:
>
>> +1, this is really helpful. We should make the SQL configurations
>> consistent and more readable.
>>
>> On Wed, Feb 12, 2020 at 3:33 PM Rubén Berenguel <rb...@gmail.com>
>> wrote:
>>
>>> I love it, it will make configs easier to read and write. Thanks
>>> Wenchen.
>>>
>>> R
>>>
>>> On 13 Feb 2020, at 00:15, Dongjoon Hyun <do...@gmail.com> wrote:
>>>
>>> 
>>> Thank you, Wenchen.
>>>
>>> The new policy looks clear to me. +1 for the explicit policy.
>>>
>>> So, are we going to revise the existing conf names before 3.0.0 release?
>>>
>>> Or, is it applied to new up-coming configurations from now?
>>>
>>> Bests,
>>> Dongjoon.
>>>
>>> On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan <cl...@gmail.com> wrote:
>>>
>>>> Hi all,
>>>>
>>>> I'd like to discuss the naming policy of Spark configs, as for now it
>>>> depends on personal preference which leads to inconsistent namings.
>>>>
>>>> In general, the config name should be a noun that describes its meaning
>>>> clearly.
>>>> Good examples:
>>>> spark.sql.session.timeZone
>>>> spark.sql.streaming.continuous.executorQueueSize
>>>> spark.sql.statistics.histogram.numBins
>>>> Bad examples:
>>>> spark.sql.defaultSizeInBytes (default size for what?)
>>>>
>>>> Also note that, config name has many parts, joined by dots. Each part
>>>> is a namespace. Don't create namespace unnecessarily.
>>>> Good example:
>>>> spark.sql.execution.rangeExchange.sampleSizePerPartition
>>>> spark.sql.execution.arrow.maxRecordsPerBatch
>>>> Bad examples:
>>>> spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a useful
>>>> namespace, better to be .buffer.inMemoryThreshold)
>>>>
>>>> For a big feature, usually we need to create an umbrella config to turn
>>>> it on/off, and other configs for fine-grained controls. These configs
>>>> should share the same namespace, and the umbrella config should be named
>>>> like featureName.enabled. For example:
>>>> spark.sql.cbo.enabled
>>>> spark.sql.cbo.starSchemaDetection
>>>> spark.sql.cbo.starJoinFTRatio
>>>> spark.sql.cbo.joinReorder.enabled
>>>> spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good
>>>> namespace)
>>>> spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good
>>>> namespace)
>>>>
>>>> For boolean configs, in general it should end with a verb, e.g.
>>>> spark.sql.join.preferSortMergeJoin. If the config is for a feature and
>>>> you can't find a good verb for the feature, featureName.enabled is
>>>> also good.
>>>>
>>>> I'll update https://spark.apache.org/contributing.html after we reach
>>>> a consensus here. Any comments are welcome!
>>>>
>>>> Thanks,
>>>> Wenchen
>>>>
>>>>
>>>>

Re: [DISCUSS] naming policy of Spark configs

Posted by Hyukjin Kwon <gu...@gmail.com>.
+1.

2020년 2월 13일 (목) 오전 9:30, Gengliang Wang <ge...@databricks.com>님이
작성:

> +1, this is really helpful. We should make the SQL configurations
> consistent and more readable.
>
> On Wed, Feb 12, 2020 at 3:33 PM Rubén Berenguel <rb...@gmail.com>
> wrote:
>
>> I love it, it will make configs easier to read and write. Thanks Wenchen.
>>
>> R
>>
>> On 13 Feb 2020, at 00:15, Dongjoon Hyun <do...@gmail.com> wrote:
>>
>> 
>> Thank you, Wenchen.
>>
>> The new policy looks clear to me. +1 for the explicit policy.
>>
>> So, are we going to revise the existing conf names before 3.0.0 release?
>>
>> Or, is it applied to new up-coming configurations from now?
>>
>> Bests,
>> Dongjoon.
>>
>> On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan <cl...@gmail.com> wrote:
>>
>>> Hi all,
>>>
>>> I'd like to discuss the naming policy of Spark configs, as for now it
>>> depends on personal preference which leads to inconsistent namings.
>>>
>>> In general, the config name should be a noun that describes its meaning
>>> clearly.
>>> Good examples:
>>> spark.sql.session.timeZone
>>> spark.sql.streaming.continuous.executorQueueSize
>>> spark.sql.statistics.histogram.numBins
>>> Bad examples:
>>> spark.sql.defaultSizeInBytes (default size for what?)
>>>
>>> Also note that, config name has many parts, joined by dots. Each part is
>>> a namespace. Don't create namespace unnecessarily.
>>> Good example:
>>> spark.sql.execution.rangeExchange.sampleSizePerPartition
>>> spark.sql.execution.arrow.maxRecordsPerBatch
>>> Bad examples:
>>> spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a useful
>>> namespace, better to be .buffer.inMemoryThreshold)
>>>
>>> For a big feature, usually we need to create an umbrella config to turn
>>> it on/off, and other configs for fine-grained controls. These configs
>>> should share the same namespace, and the umbrella config should be named
>>> like featureName.enabled. For example:
>>> spark.sql.cbo.enabled
>>> spark.sql.cbo.starSchemaDetection
>>> spark.sql.cbo.starJoinFTRatio
>>> spark.sql.cbo.joinReorder.enabled
>>> spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good
>>> namespace)
>>> spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good
>>> namespace)
>>>
>>> For boolean configs, in general it should end with a verb, e.g.
>>> spark.sql.join.preferSortMergeJoin. If the config is for a feature and
>>> you can't find a good verb for the feature, featureName.enabled is also
>>> good.
>>>
>>> I'll update https://spark.apache.org/contributing.html after we reach a
>>> consensus here. Any comments are welcome!
>>>
>>> Thanks,
>>> Wenchen
>>>
>>>
>>>

Re: [DISCUSS] naming policy of Spark configs

Posted by Gengliang Wang <ge...@databricks.com>.
+1, this is really helpful. We should make the SQL configurations
consistent and more readable.

On Wed, Feb 12, 2020 at 3:33 PM Rubén Berenguel <rb...@gmail.com>
wrote:

> I love it, it will make configs easier to read and write. Thanks Wenchen.
>
> R
>
> On 13 Feb 2020, at 00:15, Dongjoon Hyun <do...@gmail.com> wrote:
>
> 
> Thank you, Wenchen.
>
> The new policy looks clear to me. +1 for the explicit policy.
>
> So, are we going to revise the existing conf names before 3.0.0 release?
>
> Or, is it applied to new up-coming configurations from now?
>
> Bests,
> Dongjoon.
>
> On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan <cl...@gmail.com> wrote:
>
>> Hi all,
>>
>> I'd like to discuss the naming policy of Spark configs, as for now it
>> depends on personal preference which leads to inconsistent namings.
>>
>> In general, the config name should be a noun that describes its meaning
>> clearly.
>> Good examples:
>> spark.sql.session.timeZone
>> spark.sql.streaming.continuous.executorQueueSize
>> spark.sql.statistics.histogram.numBins
>> Bad examples:
>> spark.sql.defaultSizeInBytes (default size for what?)
>>
>> Also note that, config name has many parts, joined by dots. Each part is
>> a namespace. Don't create namespace unnecessarily.
>> Good example:
>> spark.sql.execution.rangeExchange.sampleSizePerPartition
>> spark.sql.execution.arrow.maxRecordsPerBatch
>> Bad examples:
>> spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a useful
>> namespace, better to be .buffer.inMemoryThreshold)
>>
>> For a big feature, usually we need to create an umbrella config to turn
>> it on/off, and other configs for fine-grained controls. These configs
>> should share the same namespace, and the umbrella config should be named
>> like featureName.enabled. For example:
>> spark.sql.cbo.enabled
>> spark.sql.cbo.starSchemaDetection
>> spark.sql.cbo.starJoinFTRatio
>> spark.sql.cbo.joinReorder.enabled
>> spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good namespace)
>> spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good
>> namespace)
>>
>> For boolean configs, in general it should end with a verb, e.g.
>> spark.sql.join.preferSortMergeJoin. If the config is for a feature and
>> you can't find a good verb for the feature, featureName.enabled is also
>> good.
>>
>> I'll update https://spark.apache.org/contributing.html after we reach a
>> consensus here. Any comments are welcome!
>>
>> Thanks,
>> Wenchen
>>
>>
>>

Re: [DISCUSS] naming policy of Spark configs

Posted by Rubén Berenguel <rb...@gmail.com>.
I love it, it will make configs easier to read and write. Thanks Wenchen. 

R

> On 13 Feb 2020, at 00:15, Dongjoon Hyun <do...@gmail.com> wrote:
> 
> 
> Thank you, Wenchen.
> 
> The new policy looks clear to me. +1 for the explicit policy.
> 
> So, are we going to revise the existing conf names before 3.0.0 release?
> 
> Or, is it applied to new up-coming configurations from now?
> 
> Bests,
> Dongjoon.
> 
>> On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan <cl...@gmail.com> wrote:
>> Hi all,
>> 
>> I'd like to discuss the naming policy of Spark configs, as for now it depends on personal preference which leads to inconsistent namings.
>> 
>> In general, the config name should be a noun that describes its meaning clearly.
>> Good examples:
>> spark.sql.session.timeZone
>> spark.sql.streaming.continuous.executorQueueSize
>> spark.sql.statistics.histogram.numBins
>> Bad examples:
>> spark.sql.defaultSizeInBytes (default size for what?)
>> 
>> Also note that, config name has many parts, joined by dots. Each part is a namespace. Don't create namespace unnecessarily.
>> Good example:
>> spark.sql.execution.rangeExchange.sampleSizePerPartition
>> spark.sql.execution.arrow.maxRecordsPerBatch
>> Bad examples:
>> spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a useful namespace, better to be .buffer.inMemoryThreshold)
>> 
>> For a big feature, usually we need to create an umbrella config to turn it on/off, and other configs for fine-grained controls. These configs should share the same namespace, and the umbrella config should be named like featureName.enabled. For example:
>> spark.sql.cbo.enabled
>> spark.sql.cbo.starSchemaDetection
>> spark.sql.cbo.starJoinFTRatio
>> spark.sql.cbo.joinReorder.enabled
>> spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good namespace)
>> spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good namespace)
>> 
>> For boolean configs, in general it should end with a verb, e.g. spark.sql.join.preferSortMergeJoin. If the config is for a feature and you can't find a good verb for the feature, featureName.enabled is also good.
>> 
>> I'll update https://spark.apache.org/contributing.html after we reach a consensus here. Any comments are welcome!
>> 
>> Thanks,
>> Wenchen
>> 
>> 

Re: [DISCUSS] naming policy of Spark configs

Posted by Reynold Xin <rx...@databricks.com>.
This is really cool. We should also be more opinionated about how we specify time and intervals.

On Wed, Feb 12, 2020 at 3:15 PM, Dongjoon Hyun < dongjoon.hyun@gmail.com > wrote:

> 
> Thank you, Wenchen.
> 
> 
> The new policy looks clear to me. +1 for the explicit policy.
> 
> 
> So, are we going to revise the existing conf names before 3.0.0 release?
> 
> 
> Or, is it applied to new up-coming configurations from now?
> 
> 
> Bests,
> Dongjoon.
> 
> On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan < cloud0fan@ gmail. com (
> cloud0fan@gmail.com ) > wrote:
> 
> 
>> Hi all,
>> 
>> 
>> I'd like to discuss the naming policy of Spark configs, as for now it
>> depends on personal preference which leads to inconsistent namings.
>> 
>> 
>> In general, the config name should be a noun that describes its meaning
>> clearly.
>> Good examples:
>> spark.sql.session.timeZone
>> 
>> spark.sql.streaming.continuous.executorQueueSize
>> 
>> spark.sql.statistics.histogram.numBins
>> 
>> Bad examples:
>> spark.sql.defaultSizeInBytes (default size for what?)
>> 
>> 
>> 
>> Also note that, config name has many parts, joined by dots. Each part is a
>> namespace. Don't create namespace unnecessarily.
>> Good example:
>> spark.sql.execution.rangeExchange.sampleSizePerPartition
>> 
>> spark.sql.execution.arrow.maxRecordsPerBatch
>> 
>> Bad examples:
>> spark. sql. windowExec. buffer. in. memory. threshold (
>> http://spark.sql.windowexec.buffer.in.memory.threshold/ ) (" in" is not a
>> useful namespace, better to be.buffer.inMemoryThreshold )
>> 
>> 
>> 
>> For a big feature, usually we need to create an umbrella config to turn it
>> on/off, and other configs for fine-grained controls. These configs should
>> share the same namespace, and the umbrella config should be named like featureName.enabled
>> . For example:
>> spark.sql.cbo.enabled
>> 
>> spark.sql.cbo.starSchemaDetection
>> 
>> spark.sql.cbo.starJoinFTRatio
>> spark.sql.cbo.joinReorder.enabled
>> spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good namespace)
>> 
>> spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good namespace)
>> 
>> 
>> 
>> 
>> For boolean configs, in general it should end with a verb, e.g. spark.sql.join.preferSortMergeJoin
>> . If the config is for a feature and you can't find a good verb for the
>> feature, featureName.enabled is also good.
>> 
>> 
>> I'll update https:/ / spark. apache. org/ contributing. html (
>> https://spark.apache.org/contributing.html ) after we reach a consensus
>> here. Any comments are welcome!
>> 
>> 
>> Thanks,
>> Wenchen
>> 
> 
>

Re: [DISCUSS] naming policy of Spark configs

Posted by Dongjoon Hyun <do...@gmail.com>.
Thank you, Wenchen.

The new policy looks clear to me. +1 for the explicit policy.

So, are we going to revise the existing conf names before 3.0.0 release?

Or, is it applied to new up-coming configurations from now?

Bests,
Dongjoon.

On Wed, Feb 12, 2020 at 7:43 AM Wenchen Fan <cl...@gmail.com> wrote:

> Hi all,
>
> I'd like to discuss the naming policy of Spark configs, as for now it
> depends on personal preference which leads to inconsistent namings.
>
> In general, the config name should be a noun that describes its meaning
> clearly.
> Good examples:
> spark.sql.session.timeZone
> spark.sql.streaming.continuous.executorQueueSize
> spark.sql.statistics.histogram.numBins
> Bad examples:
> spark.sql.defaultSizeInBytes (default size for what?)
>
> Also note that, config name has many parts, joined by dots. Each part is a
> namespace. Don't create namespace unnecessarily.
> Good example:
> spark.sql.execution.rangeExchange.sampleSizePerPartition
> spark.sql.execution.arrow.maxRecordsPerBatch
> Bad examples:
> spark.sql.windowExec.buffer.in.memory.threshold ("in" is not a useful
> namespace, better to be .buffer.inMemoryThreshold)
>
> For a big feature, usually we need to create an umbrella config to turn it
> on/off, and other configs for fine-grained controls. These configs should
> share the same namespace, and the umbrella config should be named like
> featureName.enabled. For example:
> spark.sql.cbo.enabled
> spark.sql.cbo.starSchemaDetection
> spark.sql.cbo.starJoinFTRatio
> spark.sql.cbo.joinReorder.enabled
> spark.sql.cbo.joinReorder.dp.threshold (BTW "dp" is not a good namespace)
> spark.sql.cbo.joinReorder.card.weight (BTW "card" is not a good namespace)
>
> For boolean configs, in general it should end with a verb, e.g.
> spark.sql.join.preferSortMergeJoin. If the config is for a feature and
> you can't find a good verb for the feature, featureName.enabled is also
> good.
>
> I'll update https://spark.apache.org/contributing.html after we reach a
> consensus here. Any comments are welcome!
>
> Thanks,
> Wenchen
>
>
>